diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 12686b4737f1..52e48c505f17 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -270,7 +270,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { n1.QueryRow(t, `SELECT id from system.namespace2 WHERE name='test'`).Scan(&tableID) tablePrefix := keys.MustAddr(keys.SystemSQLCodec.TablePrefix(tableID)) n4Cache := tc.Server(3).DistSenderI().(*kvcoord.DistSender).RangeDescriptorCache() - entry := n4Cache.GetCached(tablePrefix, false /* inverted */) + entry := n4Cache.GetCached(ctx, tablePrefix, false /* inverted */) require.NotNil(t, entry) require.False(t, entry.Lease().Empty()) require.Equal(t, roachpb.StoreID(1), entry.Lease().Replica.StoreID) @@ -291,7 +291,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) { rec := <-recCh require.False(t, kv.OnlyFollowerReads(rec), "query was not served through follower reads: %s", rec) // Check that the cache was properly updated. - entry = n4Cache.GetCached(tablePrefix, false /* inverted */) + entry = n4Cache.GetCached(ctx, tablePrefix, false /* inverted */) require.NotNil(t, entry) require.False(t, entry.Lease().Empty()) require.Equal(t, roachpb.StoreID(1), entry.Lease().Replica.StoreID) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 8d963cb5ca1b..771467f1d540 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -578,7 +578,7 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { now, thresh, policy, gc.NoopGCer{}, func(_ context.Context, _ []roachpb.Intent) error { return nil }, - func(_ context.Context, _ *roachpb.Transaction, _ []roachpb.LockUpdate) error { return nil }, + func(_ context.Context, _ *roachpb.Transaction) error { return nil }, ) if err != nil { return err diff --git a/pkg/geo/bbox.go b/pkg/geo/bbox.go index 681186ff3937..9f17b4b1b688 100644 --- a/pkg/geo/bbox.go +++ b/pkg/geo/bbox.go @@ -48,15 +48,15 @@ func (b *CartesianBoundingBox) Repr() string { } // ParseCartesianBoundingBox parses a box2d string into a bounding box. -func ParseCartesianBoundingBox(s string) (*CartesianBoundingBox, error) { - b := &CartesianBoundingBox{} +func ParseCartesianBoundingBox(s string) (CartesianBoundingBox, error) { + b := CartesianBoundingBox{} var prefix string numScanned, err := fmt.Sscanf(s, "%3s(%f %f,%f %f)", &prefix, &b.LoX, &b.LoY, &b.HiX, &b.HiY) if err != nil { - return nil, errors.Wrapf(err, "error parsing box2d") + return b, errors.Wrapf(err, "error parsing box2d") } if numScanned != 5 || strings.ToLower(prefix) != "box" { - return nil, errors.Newf("expected format 'box(min_x min_y,max_x max_y)'") + return b, errors.Newf("expected format 'box(min_x min_y,max_x max_y)'") } return b, nil } diff --git a/pkg/geo/bbox_test.go b/pkg/geo/bbox_test.go index 814408777dc6..6220ecc76ef1 100644 --- a/pkg/geo/bbox_test.go +++ b/pkg/geo/bbox_test.go @@ -23,12 +23,12 @@ import ( func TestParseCartesianBoundingBox(t *testing.T) { testCases := []struct { s string - expected *CartesianBoundingBox + expected CartesianBoundingBox expectedError bool }{ { s: "box(1 2,3 4)", - expected: &CartesianBoundingBox{ + expected: CartesianBoundingBox{ BoundingBox: geopb.BoundingBox{ LoX: 1, LoY: 2, @@ -39,7 +39,7 @@ func TestParseCartesianBoundingBox(t *testing.T) { }, { s: "BOX(1 2,3 4)", - expected: &CartesianBoundingBox{ + expected: CartesianBoundingBox{ BoundingBox: geopb.BoundingBox{ LoX: 1, LoY: 2, diff --git a/pkg/geo/geomfn/topology_operations.go b/pkg/geo/geomfn/topology_operations.go index 585b1c24a7ee..24451415bc8c 100644 --- a/pkg/geo/geomfn/topology_operations.go +++ b/pkg/geo/geomfn/topology_operations.go @@ -13,7 +13,6 @@ package geomfn import ( "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/geo/geos" - "github.com/cockroachdb/errors" ) // Centroid returns the Centroid of a given Geometry. @@ -26,13 +25,10 @@ func Centroid(g *geo.Geometry) (*geo.Geometry, error) { } // ClipByRect clips a given Geometry by the given BoundingBox. -func ClipByRect(g *geo.Geometry, b *geo.CartesianBoundingBox) (*geo.Geometry, error) { +func ClipByRect(g *geo.Geometry, b geo.CartesianBoundingBox) (*geo.Geometry, error) { if g.Empty() { return g, nil } - if b == nil { - return nil, errors.Newf("expected not null bounding box") - } clipByRectEWKB, err := geos.ClipByRect(g.EWKB(), b.LoX, b.LoY, b.HiX, b.HiY) if err != nil { return nil, err diff --git a/pkg/kv/bulk/sst_batcher.go b/pkg/kv/bulk/sst_batcher.go index 31617d5a40c3..ed29e5303e32 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -213,7 +213,7 @@ func (b *SSTBatcher) flushIfNeeded(ctx context.Context, nextKey roachpb.Key) err if k, err := keys.Addr(nextKey); err != nil { log.Warningf(ctx, "failed to get RKey for flush key lookup") } else { - r := b.rc.GetCached(k, false /* inverted */) + r := b.rc.GetCached(ctx, k, false /* inverted */) if r != nil { b.flushKey = r.Desc().EndKey.AsRawKey() log.VEventf(ctx, 3, "building sstable that will flush before %v", b.flushKey) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index 1c631d425b04..397563794acb 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -580,6 +580,7 @@ func TestImmutableBatchArgs(t *testing.T) { func TestRetryOnNotLeaseHolderError(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() recognizedLeaseHolder := testUserRangeDescriptor3Replicas.Replicas().Voters()[1] unrecognizedLeaseHolder := roachpb.ReplicaDescriptor{ @@ -632,7 +633,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { for _, tc := range tests { t.Run(tc.name, func(t *testing.T) { stopper := stop.NewStopper() - defer stopper.Stop(context.Background()) + defer stopper.Stop(ctx) clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) rpcContext := rpc.NewInsecureTestingContext(clock, stopper) @@ -678,13 +679,13 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) { ds := NewDistSender(cfg) v := roachpb.MakeValueFromString("value") put := roachpb.NewPut(roachpb.Key("a"), v) - if _, pErr := kv.SendWrapped(context.Background(), ds, put); !testutils.IsPError(pErr, "boom") { + if _, pErr := kv.SendWrapped(ctx, ds, put); !testutils.IsPError(pErr, "boom") { t.Fatalf("unexpected error: %v", pErr) } if first { t.Fatal("the request did not retry") } - rng := ds.rangeCache.GetCached(testUserRangeDescriptor.StartKey, false /* inverted */) + rng := ds.rangeCache.GetCached(ctx, testUserRangeDescriptor.StartKey, false /* inverted */) require.NotNil(t, rng) if tc.expLeaseholder != nil { @@ -892,7 +893,7 @@ func TestDistSenderDownNodeEvictLeaseholder(t *testing.T) { t.Errorf("contacted n1: %t, contacted n2: %t", contacted1, contacted2) } - rng := ds.rangeCache.GetCached(testUserRangeDescriptor.StartKey, false /* inverted */) + rng := ds.rangeCache.GetCached(ctx, testUserRangeDescriptor.StartKey, false /* inverted */) require.Equal(t, roachpb.StoreID(2), rng.Lease().Replica.StoreID) } @@ -1162,7 +1163,7 @@ func TestEvictCacheOnError(t *testing.T) { if _, pErr := kv.SendWrapped(ctx, ds, put); pErr != nil && !testutils.IsPError(pErr, errString) && !testutils.IsError(pErr.GoError(), ctx.Err().Error()) { t.Errorf("put encountered unexpected error: %s", pErr) } - rng := ds.rangeCache.GetCached(testUserRangeDescriptor.StartKey, false /* inverted */) + rng := ds.rangeCache.GetCached(ctx, testUserRangeDescriptor.StartKey, false /* inverted */) if tc.shouldClearReplica { require.Nil(t, rng) } else if tc.shouldClearLeaseHolder { @@ -1674,8 +1675,7 @@ func TestDistSenderDescriptorUpdatesOnSuccessfulRPCs(t *testing.T) { // Check that the cache has the updated descriptor returned by the RPC. for _, ri := range tc { rk := ri.Desc.StartKey - entry, err := ds.rangeCache.Lookup(ctx, rk) - require.NoError(t, err) + entry := ds.rangeCache.GetCached(ctx, rk, false /* inverted */) require.NotNil(t, entry) require.Equal(t, &ri.Desc, entry.Desc()) if ri.Lease.Empty() { @@ -1746,7 +1746,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { if len(seen) == 1 { // Pretend that this replica is the leaseholder in the cache to verify // that the response evicts it. - rng := ds.rangeCache.GetCached(descriptor.StartKey, false /* inverse */) + rng := ds.rangeCache.GetCached(ctx, descriptor.StartKey, false /* inverse */) ds.rangeCache.Insert(ctx, roachpb.RangeInfo{ Desc: *rng.Desc(), Lease: roachpb.Lease{Replica: ba.Replica}, @@ -1783,7 +1783,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { t.Fatal(err) } - rng := ds.rangeCache.GetCached(descriptor.StartKey, false /* inverted */) + rng := ds.rangeCache.GetCached(ctx, descriptor.StartKey, false /* inverted */) require.NotNil(t, rng) require.Equal(t, leaseholderStoreID, rng.Lease().Replica.StoreID) } @@ -3406,7 +3406,7 @@ func TestCanSendToFollower(t *testing.T) { // we've had where we were always updating the leaseholder on successful // RPCs because we erroneously assumed that a success must come from the // leaseholder. - rng := ds.rangeCache.GetCached(testUserRangeDescriptor.StartKey, false /* inverted */) + rng := ds.rangeCache.GetCached(ctx, testUserRangeDescriptor.StartKey, false /* inverted */) require.NotNil(t, rng) require.NotNil(t, rng.Lease()) require.Equal(t, roachpb.StoreID(2), rng.Lease().Replica.StoreID) @@ -3571,7 +3571,7 @@ func TestEvictMetaRange(t *testing.T) { } // Verify that there is one meta2 cached range. - cachedRange := ds.rangeCache.GetCached(keys.RangeMetaKey(roachpb.RKey("a")), false) + cachedRange := ds.rangeCache.GetCached(ctx, keys.RangeMetaKey(roachpb.RKey("a")), false) if !cachedRange.Desc().StartKey.Equal(keys.Meta2Prefix) || !cachedRange.Desc().EndKey.Equal(testMetaEndKey) { t.Fatalf("expected cached meta2 range to be [%s, %s), actual [%s, %s)", keys.Meta2Prefix, testMetaEndKey, cachedRange.Desc().StartKey, cachedRange.Desc().EndKey) @@ -3586,12 +3586,12 @@ func TestEvictMetaRange(t *testing.T) { } // Verify that there are two meta2 cached ranges. - cachedRange = ds.rangeCache.GetCached(keys.RangeMetaKey(roachpb.RKey("a")), false) + cachedRange = ds.rangeCache.GetCached(ctx, keys.RangeMetaKey(roachpb.RKey("a")), false) if !cachedRange.Desc().StartKey.Equal(keys.Meta2Prefix) || !cachedRange.Desc().EndKey.Equal(splitKey) { t.Fatalf("expected cached meta2 range to be [%s, %s), actual [%s, %s)", keys.Meta2Prefix, splitKey, cachedRange.Desc().StartKey, cachedRange.Desc().EndKey) } - cachedRange = ds.rangeCache.GetCached(keys.RangeMetaKey(roachpb.RKey("b")), false) + cachedRange = ds.rangeCache.GetCached(ctx, keys.RangeMetaKey(roachpb.RKey("b")), false) if !cachedRange.Desc().StartKey.Equal(splitKey) || !cachedRange.Desc().EndKey.Equal(testMetaEndKey) { t.Fatalf("expected cached meta2 range to be [%s, %s), actual [%s, %s)", splitKey, testMetaEndKey, cachedRange.Desc().StartKey, cachedRange.Desc().EndKey) @@ -4139,7 +4139,7 @@ func TestSendToReplicasSkipsStaleReplicas(t *testing.T) { _, err = ds.sendToReplicas(ctx, ba, tok, false /* withCommit */) require.IsType(t, sendError{}, err) require.Regexp(t, "NotLeaseHolderError", err) - cached := rc.GetCached(desc.StartKey, false /* inverted */) + cached := rc.GetCached(ctx, desc.StartKey, false /* inverted */) if tc.expLeaseholder == 0 { // Check that the descriptor was removed from the cache. require.Nil(t, cached) diff --git a/pkg/kv/kvclient/kvcoord/range_cache.go b/pkg/kv/kvclient/kvcoord/range_cache.go index caa65dfbbc98..6de6588d9a30 100644 --- a/pkg/kv/kvclient/kvcoord/range_cache.go +++ b/pkg/kv/kvclient/kvcoord/range_cache.go @@ -41,7 +41,7 @@ import ( // RangeCache. type rangeCacheKey roachpb.RKey -var maxCacheKey interface{} = rangeCacheKey(roachpb.RKeyMax) +var minCacheKey interface{} = rangeCacheKey(roachpb.RKeyMin) func (a rangeCacheKey) String() string { return roachpb.Key(a).String() @@ -535,7 +535,7 @@ func (rdc *RangeDescriptorCache) Lookup( } // GetCachedOverlapping returns all the cached entries which overlap a given -// span. +// span [Key, EndKey). The results are sorted ascendingly. func (rdc *RangeDescriptorCache) GetCachedOverlapping( ctx context.Context, span roachpb.RSpan, ) []kvbase.RangeCacheEntry { @@ -552,23 +552,24 @@ func (rdc *RangeDescriptorCache) GetCachedOverlapping( func (rdc *RangeDescriptorCache) getCachedOverlappingRLocked( ctx context.Context, span roachpb.RSpan, ) []*cache.Entry { - start := rangeCacheKey(keys.RangeMetaKey(span.Key).Next()) var res []*cache.Entry - // We iterate from the range meta key after RangeMetaKey(desc.StartKey) to the - // end of the key space and we stop when we hit a descriptor to the right of - // span. Notice the Next() we use for the start key to avoid clearing the - // descriptor that ends where span starts: for example, if we are inserting - // ["b", "c"), we should not evict ["a", "b"). - rdc.rangeCache.cache.DoRangeEntry(func(e *cache.Entry) (exit bool) { - cached := rdc.getValue(e) - // Stop when we hit a descriptor to the right of span. The end key is - // exclusive, so if span is [a,b) and we hit [b,c), we stop. - if span.EndKey.Compare(cached.Desc().StartKey) <= 0 { + rdc.rangeCache.cache.DoRangeReverseEntry(func(e *cache.Entry) (exit bool) { + desc := rdc.getValue(e).Desc() + if desc.StartKey.Equal(span.EndKey) { + // Skip over descriptor starting at the end key, who'd supposed to be exclusive. + return false + } + // Stop when we get to a lower range. + if desc.EndKey.Compare(span.Key) <= 0 { return true } res = append(res, e) return false // continue iterating - }, start, maxCacheKey) + }, rangeCacheKey(span.EndKey), minCacheKey) + // Invert the results so the get sorted ascendingly. + for i, j := 0, len(res)-1; i < j; i, j = i+1, j-1 { + res[i], res[j] = res[j], res[i] + } return res } @@ -622,7 +623,7 @@ func (rdc *RangeDescriptorCache) tryLookup( ctx context.Context, key roachpb.RKey, evictToken EvictionToken, useReverseScan bool, ) (EvictionToken, error) { rdc.rangeCache.RLock() - if entry, _ := rdc.getCachedLocked(key, useReverseScan); entry != nil { + if entry, _ := rdc.getCachedRLocked(ctx, key, useReverseScan); entry != nil { rdc.rangeCache.RUnlock() returnToken := rdc.makeEvictionToken(entry, nil /* nextDesc */) return returnToken, nil @@ -817,7 +818,7 @@ func (rdc *RangeDescriptorCache) EvictByKey(ctx context.Context, descKey roachpb rdc.rangeCache.Lock() defer rdc.rangeCache.Unlock() - cachedDesc, entry := rdc.getCachedLocked(descKey, false /* inverted */) + cachedDesc, entry := rdc.getCachedRLocked(ctx, descKey, false /* inverted */) if cachedDesc == nil { return false } @@ -839,7 +840,7 @@ func (rdc *RangeDescriptorCache) EvictByKey(ctx context.Context, descKey roachpb func (rdc *RangeDescriptorCache) evictLocked( ctx context.Context, entry *rangeCacheEntry, ) (ok bool, updatedEntry *rangeCacheEntry) { - cachedEntry, rawEntry := rdc.getCachedLocked(entry.desc.StartKey, false /* inverted */) + cachedEntry, rawEntry := rdc.getCachedRLocked(ctx, entry.desc.StartKey, false /* inverted */) if cachedEntry != entry { if cachedEntry != nil && descsCompatible(cachedEntry.Desc(), entry.Desc()) { return false, cachedEntry @@ -868,10 +869,12 @@ func (rdc *RangeDescriptorCache) mustEvictLocked(ctx context.Context, entry *ran // `inverted` determines the behavior at the range boundary: If set to true // and `key` is the EndKey and StartKey of two adjacent ranges, the first range // is returned instead of the second (which technically contains the given key). -func (rdc *RangeDescriptorCache) GetCached(key roachpb.RKey, inverted bool) kvbase.RangeCacheEntry { +func (rdc *RangeDescriptorCache) GetCached( + ctx context.Context, key roachpb.RKey, inverted bool, +) kvbase.RangeCacheEntry { rdc.rangeCache.RLock() defer rdc.rangeCache.RUnlock() - entry, _ := rdc.getCachedLocked(key, inverted) + entry, _ := rdc.getCachedRLocked(ctx, key, inverted) if entry == nil { // This return avoids boxing a nil into a non-nil iface. return nil @@ -879,29 +882,47 @@ func (rdc *RangeDescriptorCache) GetCached(key roachpb.RKey, inverted bool) kvba return kvbase.RangeCacheEntry(entry) } -// getCachedLocked is like GetCached, but it assumes that the caller holds a +// getCachedRLocked is like GetCached, but it assumes that the caller holds a // read lock on rdc.rangeCache. // // In addition to GetCached, it also returns an internal cache Entry that can be // used for descriptor eviction. -func (rdc *RangeDescriptorCache) getCachedLocked( - key roachpb.RKey, inverted bool, +func (rdc *RangeDescriptorCache) getCachedRLocked( + ctx context.Context, key roachpb.RKey, inverted bool, ) (*rangeCacheEntry, *cache.Entry) { - // The cache is indexed using the end-key of the range, but the - // end-key is non-inverted by default. - var metaKey roachpb.RKey + // rawEntry will be the range containing key, or the first cached entry around + // key, in the direction indicated by inverted. + var rawEntry *cache.Entry if !inverted { - metaKey = keys.RangeMetaKey(key.Next()) + var ok bool + rawEntry, ok = rdc.rangeCache.cache.FloorEntry(rangeCacheKey(key)) + if !ok { + return nil, nil + } } else { - metaKey = keys.RangeMetaKey(key) + rdc.rangeCache.cache.DoRangeReverseEntry(func(e *cache.Entry) bool { + startKey := roachpb.RKey(e.Key.(rangeCacheKey)) + if key.Equal(startKey) { + // DoRangeReverseEntry is inclusive on the higher key. We're iterating + // backwards and we got a range that starts at key. We're not interested + // in this range; we're interested in the range before it that ends at + // key. + return false // continue iterating + } + rawEntry = e + return true + }, rangeCacheKey(key), minCacheKey) + // DoRangeReverseEntry is exclusive on the "to" part, so we need to check + // manually if there's an entry for RKeyMin. + if rawEntry == nil { + rawEntry, _ = rdc.rangeCache.cache.FloorEntry(minCacheKey) + } } - rawEntry, ok := rdc.rangeCache.cache.CeilEntry(rangeCacheKey(metaKey)) - if !ok { + if rawEntry == nil { return nil, nil } entry := rdc.getValue(rawEntry) - desc := &entry.desc containsFn := (*roachpb.RangeDescriptor).ContainsKey if inverted { @@ -909,7 +930,7 @@ func (rdc *RangeDescriptorCache) getCachedLocked( } // Return nil if the key does not belong to the range. - if !containsFn(desc, key) { + if !containsFn(entry.Desc(), key) { return nil, nil } return entry, rawEntry @@ -988,9 +1009,9 @@ func (rdc *RangeDescriptorCache) insertLockedInner( entries[i] = newerEntry continue } - rangeKey := keys.RangeMetaKey(ent.Desc().EndKey) + rangeKey := ent.Desc().StartKey if log.V(2) { - log.Infof(ctx, "adding cache entry: key=%s value=%s", rangeKey, ent) + log.Infof(ctx, "adding cache entry: value=%s", ent) } rdc.rangeCache.cache.Add(rangeCacheKey(rangeKey), ent) entries[i] = ent diff --git a/pkg/kv/kvclient/kvcoord/range_cache_test.go b/pkg/kv/kvclient/kvcoord/range_cache_test.go index 6aa6b01fe1ed..f7c4fd81020b 100644 --- a/pkg/kv/kvclient/kvcoord/range_cache_test.go +++ b/pkg/kv/kvclient/kvcoord/range_cache_test.go @@ -471,6 +471,32 @@ func TestRangeCache(t *testing.T) { db.assertLookupCountEq(t, 1, "cz") } +// Test that cache lookups by RKeyMin and derivative keys work fine. +func TestLookupByKeyMin(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + st := cluster.MakeTestingClusterSettings() + stopper := stop.NewStopper() + defer stopper.Stop(context.Background()) + cache := NewRangeDescriptorCache(st, nil, staticSize(2<<10), stopper) + startToMeta2Desc := roachpb.RangeDescriptor{ + StartKey: roachpb.RKeyMin, + EndKey: keys.RangeMetaKey(roachpb.RKey("a")), + } + cache.Insert(ctx, roachpb.RangeInfo{Desc: startToMeta2Desc}) + entMin := cache.GetCached(ctx, roachpb.RKeyMin, false /* inverted */) + require.NotNil(t, entMin) + require.NotNil(t, entMin.Desc()) + require.Equal(t, startToMeta2Desc, *entMin.Desc()) + + entNext := cache.GetCached(ctx, roachpb.RKeyMin.Next(), false /* inverted */) + require.True(t, entMin == entNext) + entNext = cache.GetCached(ctx, roachpb.RKeyMin.Next().Next(), false /* inverted */) + require.True(t, entMin == entNext) +} + // TestRangeCacheCoalescedRequests verifies that concurrent lookups for // the same key will be coalesced onto the same database lookup. func TestRangeCacheCoalescedRequests(t *testing.T) { @@ -990,13 +1016,13 @@ func TestRangeCacheClearOverlapping(t *testing.T) { curGeneration := roachpb.RangeGeneration(1) require.True(t, clearOlderOverlapping(ctx, cache, minToBDesc)) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("b"))), &rangeCacheEntry{desc: *minToBDesc}) - if desc := cache.GetCached(roachpb.RKey("b"), false); desc != nil { + if desc := cache.GetCached(ctx, roachpb.RKey("b"), false); desc != nil { t.Errorf("descriptor unexpectedly non-nil: %s", desc) } require.True(t, clearOlderOverlapping(ctx, cache, bToMaxDesc)) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKeyMax)), &rangeCacheEntry{desc: *bToMaxDesc}) - ri := cache.GetCached(roachpb.RKey("b"), false) + ri := cache.GetCached(ctx, roachpb.RKey("b"), false) require.Equal(t, bToMaxDesc, ri.Desc()) // Add default descriptor back which should remove two split descriptors. @@ -1006,7 +1032,7 @@ func TestRangeCacheClearOverlapping(t *testing.T) { require.True(t, clearOlderOverlapping(ctx, cache, &defDescCpy)) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKeyMax)), &rangeCacheEntry{desc: defDescCpy}) for _, key := range []roachpb.RKey{roachpb.RKey("a"), roachpb.RKey("b")} { - ri = cache.GetCached(key, false) + ri = cache.GetCached(ctx, key, false) require.Equal(t, &defDescCpy, ri.Desc()) } @@ -1019,7 +1045,7 @@ func TestRangeCacheClearOverlapping(t *testing.T) { } require.True(t, clearOlderOverlapping(ctx, cache, bToCDesc)) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("c"))), &rangeCacheEntry{desc: *bToCDesc}) - ri = cache.GetCached(roachpb.RKey("c"), true) + ri = cache.GetCached(ctx, roachpb.RKey("c"), true) require.Equal(t, bToCDesc, ri.Desc()) curGeneration++ @@ -1030,7 +1056,7 @@ func TestRangeCacheClearOverlapping(t *testing.T) { } require.True(t, clearOlderOverlapping(ctx, cache, aToBDesc)) cache.rangeCache.cache.Add(rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("b"))), ri) - ri = cache.GetCached(roachpb.RKey("c"), true) + ri = cache.GetCached(ctx, roachpb.RKey("c"), true) require.Equal(t, bToCDesc, ri.Desc()) } @@ -1226,77 +1252,80 @@ func TestRangeCacheClearOverlappingMeta(t *testing.T) { func TestGetCachedRangeDescriptorInverted(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() testData := []roachpb.RangeDescriptor{ + {StartKey: roachpb.RKeyMin, EndKey: roachpb.RKey("a")}, {StartKey: roachpb.RKey("a"), EndKey: roachpb.RKey("c")}, {StartKey: roachpb.RKey("c"), EndKey: roachpb.RKey("e")}, - {StartKey: roachpb.RKey("g"), EndKey: roachpb.RKey("z")}, + {StartKey: roachpb.RKey("l"), EndKey: roachpb.RKey("m")}, + {StartKey: roachpb.RKey("m"), EndKey: roachpb.RKey("z")}, } st := cluster.MakeTestingClusterSettings() stopper := stop.NewStopper() - defer stopper.Stop(context.Background()) + defer stopper.Stop(ctx) cache := NewRangeDescriptorCache(st, nil, staticSize(2<<10), stopper) for _, rd := range testData { - cache.rangeCache.cache.Add( - rangeCacheKey(keys.RangeMetaKey(rd.EndKey)), &rangeCacheEntry{desc: rd}) + cache.Insert(ctx, roachpb.RangeInfo{ + Desc: rd, + }) } testCases := []struct { queryKey roachpb.RKey - cacheKey rangeCacheKey rng *roachpb.RangeDescriptor }{ { // Check range start key. - queryKey: roachpb.RKey("a"), - cacheKey: nil, + queryKey: roachpb.RKey("l"), rng: nil, }, + { + // Check some key in first range. + queryKey: roachpb.RKey("0"), + rng: &roachpb.RangeDescriptor{StartKey: roachpb.RKeyMin, EndKey: roachpb.RKey("a")}, + }, + { + // Check end key of first range. + queryKey: roachpb.RKey("a"), + rng: &roachpb.RangeDescriptor{StartKey: roachpb.RKeyMin, EndKey: roachpb.RKey("a")}, + }, { // Check range end key. queryKey: roachpb.RKey("c"), - cacheKey: rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("c"))), rng: &roachpb.RangeDescriptor{StartKey: roachpb.RKey("a"), EndKey: roachpb.RKey("c")}, }, { // Check range middle key. queryKey: roachpb.RKey("d"), - cacheKey: rangeCacheKey(keys.RangeMetaKey(roachpb.RKey("e"))), rng: &roachpb.RangeDescriptor{StartKey: roachpb.RKey("c"), EndKey: roachpb.RKey("e")}, }, { // Check miss range key. queryKey: roachpb.RKey("f"), - cacheKey: nil, rng: nil, }, { // Check range start key with previous range miss. - queryKey: roachpb.RKey("g"), - cacheKey: nil, + queryKey: roachpb.RKey("l"), rng: nil, }, } for _, test := range testCases { - cache.rangeCache.RLock() - targetRange, entry := cache.getCachedLocked(test.queryKey, true /* inverted */) - cache.rangeCache.RUnlock() - - if test.rng == nil { - require.Nil(t, targetRange) - } else { - require.NotNil(t, targetRange) - require.Equal(t, test.rng, targetRange.Desc()) - } - var cacheKey rangeCacheKey - if entry != nil { - cacheKey = entry.Key.(rangeCacheKey) - } - if !reflect.DeepEqual(cacheKey, test.cacheKey) { - t.Fatalf("expect cache key %v, actual get %v", test.cacheKey, cacheKey) - } + t.Run("", func(t *testing.T) { + cache.rangeCache.RLock() + targetRange, _ := cache.getCachedRLocked(ctx, test.queryKey, true /* inverted */) + cache.rangeCache.RUnlock() + + if test.rng == nil { + require.Nil(t, targetRange) + } else { + require.NotNil(t, targetRange) + require.Equal(t, test.rng, targetRange.Desc()) + } + }) } } @@ -1379,7 +1408,7 @@ func TestRangeCacheGeneration(t *testing.T) { cache.Insert(ctx, roachpb.RangeInfo{Desc: *tc.insertDesc}) for index, queryKey := range tc.queryKeys { - ri := cache.GetCached(queryKey, false) + ri := cache.GetCached(ctx, queryKey, false) exp := tc.expectedDesc[index] if exp == nil { require.Nil(t, ri) @@ -1466,12 +1495,12 @@ func TestRangeCacheUpdateLease(t *testing.T) { tok, ok := tok.UpdateLease(ctx, l) require.True(t, ok) require.Equal(t, oldTok.Desc(), tok.Desc()) - ri := cache.GetCached(startKey, false /* inverted */) + ri := cache.GetCached(ctx, startKey, false /* inverted */) require.NotNil(t, ri) require.Equal(t, rep1, ri.Lease().Replica) tok = tok.ClearLease(ctx) - ri = cache.GetCached(startKey, false /* inverted */) + ri = cache.GetCached(ctx, startKey, false /* inverted */) require.NotNil(t, ri) require.True(t, ri.(*rangeCacheEntry).lease.Empty()) require.NotNil(t, tok) @@ -1485,7 +1514,7 @@ func TestRangeCacheUpdateLease(t *testing.T) { tok, ok = tok.UpdateLease(ctx, l) require.False(t, ok) require.True(t, tok.Empty()) - ri = cache.GetCached(startKey, false /* inverted */) + ri = cache.GetCached(ctx, startKey, false /* inverted */) require.Nil(t, ri) // Check that updating the lease while the cache has a newer descriptor @@ -1523,7 +1552,7 @@ func TestRangeCacheUpdateLease(t *testing.T) { tok, ok = tok.UpdateLease(ctx, &roachpb.Lease{Replica: rep3, Sequence: 4}) require.False(t, ok) require.True(t, tok.Empty()) - ri = cache.GetCached(startKey, false /* inverted */) + ri = cache.GetCached(ctx, startKey, false /* inverted */) require.Nil(t, ri) } diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 71aa5a343254..c9508d8f73b8 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -146,7 +146,7 @@ type CleanupIntentsFunc func(context.Context, []roachpb.Intent) error // transaction record, pushing the transaction first if it is // PENDING. Once all intents are resolved successfully, removes the // transaction record. -type CleanupTxnIntentsAsyncFunc func(context.Context, *roachpb.Transaction, []roachpb.LockUpdate) error +type CleanupTxnIntentsAsyncFunc func(context.Context, *roachpb.Transaction) error // Run runs garbage collection for the specified descriptor on the // provided Engine (which is not mutated). It uses the provided gcFn @@ -423,7 +423,7 @@ func processLocalKeyRange( // If the transaction needs to be pushed or there are intents to // resolve, invoke the cleanup function. if !txn.Status.IsFinalized() || len(txn.LockSpans) > 0 { - return cleanupTxnIntentsAsyncFn(ctx, txn, roachpb.AsLockUpdates(txn, txn.LockSpans)) + return cleanupTxnIntentsAsyncFn(ctx, txn) } b.FlushingAdd(ctx, key) return nil diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index 3b30e242c2f7..795458c79e8c 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -138,7 +138,7 @@ func BenchmarkRun(b *testing.B) { func(ctx context.Context, intents []roachpb.Intent) error { return nil }, - func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { + func(ctx context.Context, txn *roachpb.Transaction) error { return nil }) } @@ -207,10 +207,8 @@ func (f *fakeGCer) GC(ctx context.Context, keys []roachpb.GCRequest_GCKey) error return nil } -func (f *fakeGCer) resolveIntentsAsync( - _ context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate, -) error { - f.txnIntents = append(f.txnIntents, txnIntents{txn: txn, intents: intents}) +func (f *fakeGCer) resolveIntentsAsync(_ context.Context, txn *roachpb.Transaction) error { + f.txnIntents = append(f.txnIntents, txnIntents{txn: txn, intents: txn.LocksAsLockUpdates()}) return nil } diff --git a/pkg/kv/kvserver/gc_queue.go b/pkg/kv/kvserver/gc_queue.go index 8aad7de74616..ef75966809b0 100644 --- a/pkg/kv/kvserver/gc_queue.go +++ b/pkg/kv/kvserver/gc_queue.go @@ -467,15 +467,15 @@ func (gcq *gcQueue) process( } return err }, - func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { + func(ctx context.Context, txn *roachpb.Transaction) error { err := repl.store.intentResolver. - CleanupTxnIntentsOnGCAsync(ctx, repl.RangeID, txn, intents, gcTimestamp, + CleanupTxnIntentsOnGCAsync(ctx, repl.RangeID, txn, gcTimestamp, func(pushed, succeeded bool) { if pushed { gcq.store.metrics.GCPushTxn.Inc(1) } if succeeded { - gcq.store.metrics.GCResolveSuccess.Inc(int64(len(intents))) + gcq.store.metrics.GCResolveSuccess.Inc(int64(len(txn.LockSpans))) } }) if errors.Is(err, stop.ErrThrottled) { diff --git a/pkg/kv/kvserver/gc_queue_test.go b/pkg/kv/kvserver/gc_queue_test.go index 1ec998adfff9..d9492aa41390 100644 --- a/pkg/kv/kvserver/gc_queue_test.go +++ b/pkg/kv/kvserver/gc_queue_test.go @@ -574,7 +574,7 @@ func TestGCQueueProcess(t *testing.T) { func(ctx context.Context, intents []roachpb.Intent) error { return nil }, - func(ctx context.Context, txn *roachpb.Transaction, intents []roachpb.LockUpdate) error { + func(ctx context.Context, txn *roachpb.Transaction) error { return nil }) }() diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index e48f8b212c34..340cbd454174 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -546,7 +546,6 @@ func (ir *IntentResolver) CleanupTxnIntentsAsync( endTxns []result.EndTxnIntents, allowSyncProcessing bool, ) error { - now := ir.clock.Now() for i := range endTxns { et := &endTxns[i] // copy for goroutine if err := ir.runAsyncTask(ctx, allowSyncProcessing, func(ctx context.Context) { @@ -555,8 +554,9 @@ func (ir *IntentResolver) CleanupTxnIntentsAsync( return } defer release() - intents := roachpb.AsLockUpdates(et.Txn, et.Txn.LockSpans) - if err := ir.cleanupFinishedTxnIntents(ctx, rangeID, et.Txn, intents, now, et.Poison, nil); err != nil { + if err := ir.cleanupFinishedTxnIntents( + ctx, rangeID, et.Txn, et.Poison, nil, /* onComplete */ + ); err != nil { if ir.every.ShouldLog() { log.Warningf(ctx, "failed to cleanup transaction intents: %v", err) } @@ -600,7 +600,6 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( ctx context.Context, rangeID roachpb.RangeID, txn *roachpb.Transaction, - intents []roachpb.LockUpdate, now hlc.Timestamp, onComplete func(pushed, succeeded bool), ) error { @@ -652,11 +651,11 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( log.VErrEventf(ctx, 2, "failed to push %s, expired txn (%s): %s", txn.Status, txn, err) return } - // Get the pushed txn and update the intents slice. - txn = &b.RawResponse().Responses[0].GetInner().(*roachpb.PushTxnResponse).PusheeTxn - for i := range intents { - intents[i].SetTxn(txn) - } + // Update the txn with the result of the push, such that the intents we're about + // to resolve get a final status. + finalizedTxn := &b.RawResponse().Responses[0].GetInner().(*roachpb.PushTxnResponse).PusheeTxn + txn = txn.Clone() + txn.Update(finalizedTxn) } var onCleanupComplete func(error) if onComplete != nil { @@ -668,7 +667,7 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( // Set onComplete to nil to disable the deferred call as the call has now // been delegated to the callback passed to cleanupFinishedTxnIntents. onComplete = nil - err := ir.cleanupFinishedTxnIntents(ctx, rangeID, txn, intents, now, false /* poison */, onCleanupComplete) + err := ir.cleanupFinishedTxnIntents(ctx, rangeID, txn, false /* poison */, onCleanupComplete) if err != nil { if ir.every.ShouldLog() { log.Warningf(ctx, "failed to cleanup transaction intents: %+v", err) @@ -726,17 +725,14 @@ func (ir *IntentResolver) gcTxnRecord( return nil } -// cleanupFinishedTxnIntents cleans up extant intents owned by a single -// transaction and when all intents have been successfully resolved, the -// transaction record is GC'ed asynchronously. onComplete will be called when -// all processing has completed which is likely to be after this call returns -// in the case of success. +// cleanupFinishedTxnIntents cleans up a txn's extant intents and, when all +// intents have been successfully resolved, the transaction record is GC'ed +// asynchronously. onComplete will be called when all processing has completed +// which is likely to be after this call returns in the case of success. func (ir *IntentResolver) cleanupFinishedTxnIntents( ctx context.Context, rangeID roachpb.RangeID, txn *roachpb.Transaction, - intents []roachpb.LockUpdate, - now hlc.Timestamp, poison bool, onComplete func(error), ) (err error) { @@ -749,7 +745,7 @@ func (ir *IntentResolver) cleanupFinishedTxnIntents( }() // Resolve intents. opts := ResolveOptions{Poison: poison, MinTimestamp: txn.MinTimestamp} - if pErr := ir.ResolveIntents(ctx, intents, opts); pErr != nil { + if pErr := ir.ResolveIntents(ctx, txn.LocksAsLockUpdates(), opts); pErr != nil { return errors.Wrapf(pErr.GoError(), "failed to resolve intents") } // Run transaction record GC outside of ir.sem. @@ -769,10 +765,14 @@ func (ir *IntentResolver) cleanupFinishedTxnIntents( }) } -// ResolveOptions is used during intent resolution. It specifies whether the -// caller wants the call to block, and whether the ranges containing the intents -// are to be poisoned. +// ResolveOptions is used during intent resolution. type ResolveOptions struct { + // If set, the abort spans on the ranges containing the intents are to be + // poisoned. If the transaction that had laid down this intent is still + // running (so this resolving is performed by a pusher) and goes back to these + // ranges trying to read one of its old intents, the access will be trapped + // and the read will return an error, thus avoiding the read missing to see + // its own write. Poison bool // The original transaction timestamp from the earliest txn epoch; if // supplied, resolution of intent ranges can be optimized in some cases. diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go index 3b3fb419c016..0bb07e945936 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go @@ -53,8 +53,10 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { Clock: clock, } type testCase struct { - txn *roachpb.Transaction - intents []roachpb.LockUpdate + txn *roachpb.Transaction + // intentSpans, if set, are appended to txn.LockSpans. They'll result in + // ResolveIntent requests. + intentSpans []roachpb.Span sendFuncs *sendFuncs expectPushed bool expectSucceed bool @@ -106,13 +108,13 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { // has been pushed but that the garbage collection was not successful. { txn: txn1, - intents: []roachpb.LockUpdate{ - roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: key}), - roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), + intentSpans: []roachpb.Span{ + {Key: key}, + {Key: key, EndKey: roachpb.Key("b")}, }, sendFuncs: newSendFuncs(t, singlePushTxnSendFunc(t), - resolveIntentsSendFunc(t), + resolveIntentsSendFuncEx(t, checkTxnAborted), failSendFunc, ), expectPushed: true, @@ -126,10 +128,10 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { // that the txn has both been pushed and successfully resolved. { txn: txn1, - intents: []roachpb.LockUpdate{ - roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: key}), - roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: roachpb.Key("aa")}), - roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), + intentSpans: []roachpb.Span{ + {Key: key}, + {Key: roachpb.Key("aa")}, + {Key: key, EndKey: roachpb.Key("b")}, }, sendFuncs: func() *sendFuncs { s := newSendFuncs(t) @@ -165,9 +167,9 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { // has been pushed but that the garbage collection was not successful. { txn: txn3, - intents: []roachpb.LockUpdate{ - roachpb.MakeLockUpdate(txn3, roachpb.Span{Key: key}), - roachpb.MakeLockUpdate(txn3, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), + intentSpans: []roachpb.Span{ + {Key: key}, + {Key: key, EndKey: roachpb.Key("b")}, }, sendFuncs: newSendFuncs(t, singlePushTxnSendFunc(t), @@ -185,10 +187,10 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { // that the txn has both been pushed and successfully resolved. { txn: txn3, - intents: []roachpb.LockUpdate{ - roachpb.MakeLockUpdate(txn3, roachpb.Span{Key: key}), - roachpb.MakeLockUpdate(txn3, roachpb.Span{Key: roachpb.Key("aa")}), - roachpb.MakeLockUpdate(txn3, roachpb.Span{Key: key, EndKey: roachpb.Key("b")}), + intentSpans: []roachpb.Span{ + {Key: key}, + {Key: roachpb.Key("aa")}, + {Key: key, EndKey: roachpb.Key("b")}, }, sendFuncs: func() *sendFuncs { s := newSendFuncs(t) @@ -207,7 +209,7 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { // is no push but that the gc has occurred successfully. { txn: txn4, - intents: []roachpb.LockUpdate{}, + intentSpans: []roachpb.Span{}, sendFuncs: newSendFuncs(t, gcSendFunc(t)), expectSucceed: true, }, @@ -222,7 +224,9 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { didPush, didSucceed = pushed, succeeded close(done) } - err := ir.CleanupTxnIntentsOnGCAsync(ctx, 1, c.txn, c.intents, clock.Now(), onComplete) + txn := c.txn.Clone() + txn.LockSpans = append([]roachpb.Span{}, c.intentSpans...) + err := ir.CleanupTxnIntentsOnGCAsync(ctx, 1, txn, clock.Now(), onComplete) if err != nil { t.Fatalf("unexpected error sending async transaction") } @@ -480,7 +484,7 @@ func TestCleanupTxnIntentsAsyncWithPartialRollback(t *testing.T) { } } } - return respForResolveIntentBatch(t, ba), nil + return respForResolveIntentBatch(t, ba, dontCheckTxnStatus), nil } sf := newSendFuncs(t, sendFunc(check), @@ -832,7 +836,26 @@ func singlePushTxnSendFunc(t *testing.T) sendFunc { return pushTxnSendFunc(t, 1) } -func resolveIntentsSendFuncs(sf *sendFuncs, numIntents int, minRequests int) sendFunc { +// checkTxnStatusOpt specifies whether some mock handlers for ResolveIntent(s) +// request should assert the intent's status before resolving it, or not. +type checkTxnStatusOpt bool + +const ( + // checkTxnAborted makes the mock ResolveIntent check that the intent's txn is + // aborted (and so the intent would be discarded by the production code). + checkTxnAborted checkTxnStatusOpt = true + + // NOTE: There should be a checkTxnCommitted option, but no test currently + // uses it. + + // A bunch of tests use dontCheckTxnStatus because they take shortcuts that + // causes intents to not be cleaned with a txn that was properly finalized. + dontCheckTxnStatus checkTxnStatusOpt = false +) + +func resolveIntentsSendFuncsEx( + sf *sendFuncs, numIntents int, minRequests int, opt checkTxnStatusOpt, +) sendFunc { toResolve := int64(numIntents) reqsSeen := int64(0) var f sendFunc @@ -849,17 +872,29 @@ func resolveIntentsSendFuncs(sf *sendFuncs, numIntents int, minRequests int) sen sf.t.Errorf("expected at least %d requests to resolve %d intents, only saw %d", minRequests, numIntents, seen) } - return respForResolveIntentBatch(sf.t, ba), nil + return respForResolveIntentBatch(sf.t, ba, opt), nil } return f } -func resolveIntentsSendFunc(t *testing.T) sendFunc { +func resolveIntentsSendFuncEx(t *testing.T, checkTxnStatusOpt checkTxnStatusOpt) sendFunc { return func(ba roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { - return respForResolveIntentBatch(t, ba), nil + return respForResolveIntentBatch(t, ba, checkTxnStatusOpt), nil } } +// resolveIntentsSendFuncs is like resolveIntentsSendFuncsEx, except it never checks +// the intents' txn status. +func resolveIntentsSendFuncs(sf *sendFuncs, numIntents int, minRequests int) sendFunc { + return resolveIntentsSendFuncsEx(sf, numIntents, minRequests, dontCheckTxnStatus) +} + +// resolveIntentsSendFunc is like resolveIntentsSendFuncEx, but it never checks +// the intents' txn status. +func resolveIntentsSendFunc(t *testing.T) sendFunc { + return resolveIntentsSendFuncEx(t, dontCheckTxnStatus) +} + func failSendFunc(roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { return nil, roachpb.NewError(fmt.Errorf("boom")) } @@ -896,16 +931,24 @@ func respForPushTxnBatch(t *testing.T, ba roachpb.BatchRequest) *roachpb.BatchRe return resp } -func respForResolveIntentBatch(t *testing.T, ba roachpb.BatchRequest) *roachpb.BatchResponse { +func respForResolveIntentBatch( + t *testing.T, ba roachpb.BatchRequest, checkTxnStatusOpt checkTxnStatusOpt, +) *roachpb.BatchResponse { resp := &roachpb.BatchResponse{} + var status roachpb.TransactionStatus for _, r := range ba.Requests { - if _, ok := r.GetInner().(*roachpb.ResolveIntentRequest); ok { + if rir, ok := r.GetInner().(*roachpb.ResolveIntentRequest); ok { + status = rir.AsLockUpdate().Status resp.Add(&roachpb.ResolveIntentResponse{}) - } else if _, ok := r.GetInner().(*roachpb.ResolveIntentRangeRequest); ok { + } else if rirr, ok := r.GetInner().(*roachpb.ResolveIntentRangeRequest); ok { + status = rirr.AsLockUpdate().Status resp.Add(&roachpb.ResolveIntentRangeResponse{}) } else { t.Errorf("Unexpected request in batch for intent resolution: %T", r.GetInner()) } } + if checkTxnStatusOpt == checkTxnAborted && status != roachpb.ABORTED { + t.Errorf("expected txn to be finalized, got status: %s", status) + } return resp } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index ea6c8f80c978..46746304a5be 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -72,7 +72,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{0} + return fileDescriptor_api_f7f080cd2994bec8, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -100,7 +100,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{1} + return fileDescriptor_api_f7f080cd2994bec8, []int{1} } type ChecksumMode int32 @@ -147,7 +147,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{2} + return fileDescriptor_api_f7f080cd2994bec8, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -178,7 +178,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{3} + return fileDescriptor_api_f7f080cd2994bec8, []int{3} } type ExternalStorageProvider int32 @@ -219,7 +219,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{4} + return fileDescriptor_api_f7f080cd2994bec8, []int{4} } type MVCCFilter int32 @@ -242,7 +242,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{5} + return fileDescriptor_api_f7f080cd2994bec8, []int{5} } type ResponseHeader_ResumeReason int32 @@ -268,7 +268,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{1, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -310,7 +310,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{25, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -331,7 +331,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{0} + return fileDescriptor_api_f7f080cd2994bec8, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -402,7 +402,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{1} + return fileDescriptor_api_f7f080cd2994bec8, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -436,7 +436,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{2} + return fileDescriptor_api_f7f080cd2994bec8, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -479,7 +479,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{3} + return fileDescriptor_api_f7f080cd2994bec8, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +522,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{4} + return fileDescriptor_api_f7f080cd2994bec8, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -556,7 +556,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{5} + return fileDescriptor_api_f7f080cd2994bec8, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -638,7 +638,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{6} + return fileDescriptor_api_f7f080cd2994bec8, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -673,7 +673,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{7} + return fileDescriptor_api_f7f080cd2994bec8, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -719,7 +719,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{8} + return fileDescriptor_api_f7f080cd2994bec8, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -753,7 +753,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{9} + return fileDescriptor_api_f7f080cd2994bec8, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -793,7 +793,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{10} + return fileDescriptor_api_f7f080cd2994bec8, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -830,7 +830,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{11} + return fileDescriptor_api_f7f080cd2994bec8, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -864,7 +864,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{12} + return fileDescriptor_api_f7f080cd2994bec8, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -898,7 +898,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{13} + return fileDescriptor_api_f7f080cd2994bec8, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -950,7 +950,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{14} + return fileDescriptor_api_f7f080cd2994bec8, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +987,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{15} + return fileDescriptor_api_f7f080cd2994bec8, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1042,7 +1042,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{16} + return fileDescriptor_api_f7f080cd2994bec8, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1076,7 +1076,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{17} + return fileDescriptor_api_f7f080cd2994bec8, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1117,7 +1117,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{18} + return fileDescriptor_api_f7f080cd2994bec8, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1151,7 +1151,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{19} + return fileDescriptor_api_f7f080cd2994bec8, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1202,7 +1202,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{20} + return fileDescriptor_api_f7f080cd2994bec8, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1254,7 +1254,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{21} + return fileDescriptor_api_f7f080cd2994bec8, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1305,7 +1305,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{22} + return fileDescriptor_api_f7f080cd2994bec8, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1357,7 +1357,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{23} + return fileDescriptor_api_f7f080cd2994bec8, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1410,7 +1410,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{24} + return fileDescriptor_api_f7f080cd2994bec8, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1447,7 +1447,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{25} + return fileDescriptor_api_f7f080cd2994bec8, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1491,7 +1491,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{25, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1539,7 +1539,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{26} + return fileDescriptor_api_f7f080cd2994bec8, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1575,7 +1575,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{27} + return fileDescriptor_api_f7f080cd2994bec8, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1672,7 +1672,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{28} + return fileDescriptor_api_f7f080cd2994bec8, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1718,7 +1718,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{29} + return fileDescriptor_api_f7f080cd2994bec8, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1779,7 +1779,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{30} + return fileDescriptor_api_f7f080cd2994bec8, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1814,7 +1814,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{31} + return fileDescriptor_api_f7f080cd2994bec8, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1853,7 +1853,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{32} + return fileDescriptor_api_f7f080cd2994bec8, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1888,7 +1888,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{33} + return fileDescriptor_api_f7f080cd2994bec8, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1931,7 +1931,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{34} + return fileDescriptor_api_f7f080cd2994bec8, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1966,7 +1966,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{35} + return fileDescriptor_api_f7f080cd2994bec8, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2004,7 +2004,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{36} + return fileDescriptor_api_f7f080cd2994bec8, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2037,7 +2037,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{37} + return fileDescriptor_api_f7f080cd2994bec8, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2072,7 +2072,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{38} + return fileDescriptor_api_f7f080cd2994bec8, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2130,7 +2130,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{39} + return fileDescriptor_api_f7f080cd2994bec8, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2165,7 +2165,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{40} + return fileDescriptor_api_f7f080cd2994bec8, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2202,7 +2202,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{41} + return fileDescriptor_api_f7f080cd2994bec8, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2235,7 +2235,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{42} + return fileDescriptor_api_f7f080cd2994bec8, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2274,7 +2274,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{43} + return fileDescriptor_api_f7f080cd2994bec8, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2311,7 +2311,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{44} + return fileDescriptor_api_f7f080cd2994bec8, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2349,7 +2349,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{45} + return fileDescriptor_api_f7f080cd2994bec8, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2383,7 +2383,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{45, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2417,7 +2417,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{46} + return fileDescriptor_api_f7f080cd2994bec8, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2486,7 +2486,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{47} + return fileDescriptor_api_f7f080cd2994bec8, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2529,7 +2529,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{48} + return fileDescriptor_api_f7f080cd2994bec8, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2576,7 +2576,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{49} + return fileDescriptor_api_f7f080cd2994bec8, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2612,7 +2612,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{50} + return fileDescriptor_api_f7f080cd2994bec8, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2656,7 +2656,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{51} + return fileDescriptor_api_f7f080cd2994bec8, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2695,7 +2695,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{52} + return fileDescriptor_api_f7f080cd2994bec8, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2755,7 +2755,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{53} + return fileDescriptor_api_f7f080cd2994bec8, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2791,7 +2791,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{54} + return fileDescriptor_api_f7f080cd2994bec8, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2826,8 +2826,7 @@ type ResolveIntentRequest struct { IntentTxn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=intent_txn,json=intentTxn,proto3" json:"intent_txn"` // The status of the transaction. Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` - // Optionally poison the abort span for the transaction the intent's - // range. + // Optionally poison the abort span for the transaction on the intent's range. Poison bool `protobuf:"varint,4,opt,name=poison,proto3" json:"poison,omitempty"` // The list of ignored seqnum ranges as per the Transaction record. IgnoredSeqNums []enginepb.IgnoredSeqNumRange `protobuf:"bytes,5,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` @@ -2837,7 +2836,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{55} + return fileDescriptor_api_f7f080cd2994bec8, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2872,7 +2871,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{56} + return fileDescriptor_api_f7f080cd2994bec8, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2922,7 +2921,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{57} + return fileDescriptor_api_f7f080cd2994bec8, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2957,7 +2956,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{58} + return fileDescriptor_api_f7f080cd2994bec8, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2994,7 +2993,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{59} + return fileDescriptor_api_f7f080cd2994bec8, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3028,7 +3027,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{60} + return fileDescriptor_api_f7f080cd2994bec8, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3073,7 +3072,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{61} + return fileDescriptor_api_f7f080cd2994bec8, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3107,7 +3106,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{62} + return fileDescriptor_api_f7f080cd2994bec8, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3151,7 +3150,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{63} + return fileDescriptor_api_f7f080cd2994bec8, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3200,7 +3199,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{64} + return fileDescriptor_api_f7f080cd2994bec8, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3237,7 +3236,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{65} + return fileDescriptor_api_f7f080cd2994bec8, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3274,7 +3273,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{66} + return fileDescriptor_api_f7f080cd2994bec8, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3309,7 +3308,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{67} + return fileDescriptor_api_f7f080cd2994bec8, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3364,7 +3363,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{68} + return fileDescriptor_api_f7f080cd2994bec8, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3401,7 +3400,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{69} + return fileDescriptor_api_f7f080cd2994bec8, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3441,7 +3440,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70} + return fileDescriptor_api_f7f080cd2994bec8, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3475,7 +3474,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3508,7 +3507,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 1} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3548,7 +3547,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 2} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3587,7 +3586,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 3} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3623,7 +3622,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 4} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3662,7 +3661,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 5} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3702,7 +3701,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{70, 6} + return fileDescriptor_api_f7f080cd2994bec8, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3742,7 +3741,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{71} + return fileDescriptor_api_f7f080cd2994bec8, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3776,7 +3775,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{72} + return fileDescriptor_api_f7f080cd2994bec8, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3812,7 +3811,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{73} + return fileDescriptor_api_f7f080cd2994bec8, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3882,7 +3881,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{74} + return fileDescriptor_api_f7f080cd2994bec8, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3934,7 +3933,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{75} + return fileDescriptor_api_f7f080cd2994bec8, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3970,7 +3969,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{76} + return fileDescriptor_api_f7f080cd2994bec8, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4010,7 +4009,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{76, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4061,7 +4060,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{77} + return fileDescriptor_api_f7f080cd2994bec8, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4096,7 +4095,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{77, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4132,7 +4131,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{77, 1} + return fileDescriptor_api_f7f080cd2994bec8, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4167,7 +4166,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{78} + return fileDescriptor_api_f7f080cd2994bec8, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4205,7 +4204,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{79} + return fileDescriptor_api_f7f080cd2994bec8, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4242,7 +4241,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{80} + return fileDescriptor_api_f7f080cd2994bec8, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4275,7 +4274,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{80, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4320,7 +4319,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{81} + return fileDescriptor_api_f7f080cd2994bec8, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4358,7 +4357,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{82} + return fileDescriptor_api_f7f080cd2994bec8, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4411,7 +4410,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{83} + return fileDescriptor_api_f7f080cd2994bec8, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4445,7 +4444,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{84} + return fileDescriptor_api_f7f080cd2994bec8, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4489,7 +4488,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{85} + return fileDescriptor_api_f7f080cd2994bec8, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4523,7 +4522,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{86} + return fileDescriptor_api_f7f080cd2994bec8, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4562,7 +4561,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{87} + return fileDescriptor_api_f7f080cd2994bec8, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4596,7 +4595,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{88} + return fileDescriptor_api_f7f080cd2994bec8, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4645,7 +4644,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{89} + return fileDescriptor_api_f7f080cd2994bec8, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4694,7 +4693,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{90} + return fileDescriptor_api_f7f080cd2994bec8, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4729,7 +4728,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{91} + return fileDescriptor_api_f7f080cd2994bec8, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4771,7 +4770,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{92} + return fileDescriptor_api_f7f080cd2994bec8, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4854,7 +4853,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{93} + return fileDescriptor_api_f7f080cd2994bec8, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6304,7 +6303,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{94} + return fileDescriptor_api_f7f080cd2994bec8, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7839,7 +7838,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{95} + return fileDescriptor_api_f7f080cd2994bec8, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7877,7 +7876,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{96} + return fileDescriptor_api_f7f080cd2994bec8, []int{96} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7913,7 +7912,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{97} + return fileDescriptor_api_f7f080cd2994bec8, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7950,7 +7949,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{98} + return fileDescriptor_api_f7f080cd2994bec8, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8024,7 +8023,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{98, 0} + return fileDescriptor_api_f7f080cd2994bec8, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8062,7 +8061,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{99} + return fileDescriptor_api_f7f080cd2994bec8, []int{99} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8100,7 +8099,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{100} + return fileDescriptor_api_f7f080cd2994bec8, []int{100} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8139,7 +8138,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{101} + return fileDescriptor_api_f7f080cd2994bec8, []int{101} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8180,7 +8179,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{102} + return fileDescriptor_api_f7f080cd2994bec8, []int{102} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8221,7 +8220,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{103} + return fileDescriptor_api_f7f080cd2994bec8, []int{103} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8258,7 +8257,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{104} + return fileDescriptor_api_f7f080cd2994bec8, []int{104} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8295,7 +8294,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{105} + return fileDescriptor_api_f7f080cd2994bec8, []int{105} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8336,7 +8335,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{106} + return fileDescriptor_api_f7f080cd2994bec8, []int{106} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8376,7 +8375,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_18cf6a70fea44207, []int{107} + return fileDescriptor_api_f7f080cd2994bec8, []int{107} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -39235,9 +39234,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_18cf6a70fea44207) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_f7f080cd2994bec8) } -var fileDescriptor_api_18cf6a70fea44207 = []byte{ +var fileDescriptor_api_f7f080cd2994bec8 = []byte{ // 7832 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, 0xb5, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x90, 0xa2, 0x5a, 0x25, 0xcd, 0x0c, 0x47, 0xb3, 0x2b, 0x69, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index f9e29232c418..61e8a48fed76 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1078,8 +1078,7 @@ message ResolveIntentRequest { storage.enginepb.TxnMeta intent_txn = 2 [(gogoproto.nullable) = false]; // The status of the transaction. TransactionStatus status = 3; - // Optionally poison the abort span for the transaction the intent's - // range. + // Optionally poison the abort span for the transaction on the intent's range. bool poison = 4; // The list of ignored seqnum ranges as per the Transaction record. repeated storage.enginepb.IgnoredSeqNumRange ignored_seqnums = 5 [ diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 17278ce732be..1547d75267f2 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -88,7 +88,7 @@ func (rk RKey) AsRawKey() Key { // Less returns true if receiver < otherRK. func (rk RKey) Less(otherRK RKey) bool { - return bytes.Compare(rk, otherRK) < 0 + return rk.Compare(otherRK) < 0 } // Compare compares the two RKeys. @@ -434,7 +434,7 @@ func (v *Value) SetGeo(so geopb.SpatialObject) error { // SetBox2D encodes the specified Box2D value into the bytes field of the // receiver, sets the tag and clears the checksum. -func (v *Value) SetBox2D(b *geo.CartesianBoundingBox) { +func (v *Value) SetBox2D(b geo.CartesianBoundingBox) { v.ensureRawBytes(headerSize + 32) encoding.EncodeUint64Ascending(v.RawBytes[headerSize:headerSize], math.Float64bits(b.LoX)) encoding.EncodeUint64Ascending(v.RawBytes[headerSize+8:headerSize+8], math.Float64bits(b.HiX)) @@ -582,35 +582,35 @@ func (v Value) GetGeo() (geopb.SpatialObject, error) { // GetBox2D decodes a geo value from the bytes field of the receiver. If the // tag is not BOX2D an error will be returned. -func (v Value) GetBox2D() (*geo.CartesianBoundingBox, error) { +func (v Value) GetBox2D() (geo.CartesianBoundingBox, error) { + box := geo.CartesianBoundingBox{} if tag := v.GetTag(); tag != ValueType_BOX2D { - return nil, fmt.Errorf("value type is not %s: %s", ValueType_BOX2D, tag) + return box, fmt.Errorf("value type is not %s: %s", ValueType_BOX2D, tag) } - box := &geo.CartesianBoundingBox{} dataBytes := v.dataBytes() if len(dataBytes) != 32 { - return nil, fmt.Errorf("float64 value should be exactly 32 bytes: %d", len(dataBytes)) + return box, fmt.Errorf("float64 value should be exactly 32 bytes: %d", len(dataBytes)) } var err error var val uint64 dataBytes, val, err = encoding.DecodeUint64Ascending(dataBytes) if err != nil { - return nil, err + return box, err } box.LoX = math.Float64frombits(val) dataBytes, val, err = encoding.DecodeUint64Ascending(dataBytes) if err != nil { - return nil, err + return box, err } box.HiX = math.Float64frombits(val) dataBytes, val, err = encoding.DecodeUint64Ascending(dataBytes) if err != nil { - return nil, err + return box, err } box.LoY = math.Float64frombits(val) _, val, err = encoding.DecodeUint64Ascending(dataBytes) if err != nil { - return nil, err + return box, err } box.HiY = math.Float64frombits(val) @@ -1178,6 +1178,15 @@ func (t *Transaction) IsLocking() bool { return t.Key != nil } +// LocksAsLockUpdates turns t.LockSpans into a bunch of LockUpdates. +func (t *Transaction) LocksAsLockUpdates() []LockUpdate { + ret := make([]LockUpdate, len(t.LockSpans)) + for i, sp := range t.LockSpans { + ret[i] = MakeLockUpdate(t, sp) + } + return ret +} + // String formats transaction into human readable string. // // NOTE: When updating String(), you probably want to also update SafeMessage(). @@ -1993,22 +2002,14 @@ func MakeLockAcquisition(txn *Transaction, key Key, dur lock.Durability) LockAcq } // MakeLockUpdate makes a lock update from the given txn and span. +// +// See also txn.LocksAsLockUpdates(). func MakeLockUpdate(txn *Transaction, span Span) LockUpdate { u := LockUpdate{Span: span} u.SetTxn(txn) return u } -// AsLockUpdates takes a slice of spans and returns it as a slice of -// lock updates. -func AsLockUpdates(txn *Transaction, spans []Span) []LockUpdate { - ret := make([]LockUpdate, len(spans)) - for i := range spans { - ret[i] = MakeLockUpdate(txn, spans[i]) - } - return ret -} - // SetTxn updates the transaction details in the lock update. func (u *LockUpdate) SetTxn(txn *Transaction) { u.Txn = txn.TxnMeta diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 9fe9f669560c..0ea7f239dddc 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -1902,9 +1902,9 @@ func TestChangeReplicasTrigger_ConfChange(t *testing.T) { } } -// TestAsLockUpdates verifies that AsLockUpdates propagates all the important -// fields from a txn to each intent. -func TestAsLockUpdates(t *testing.T) { +// TestAsLockUpdates verifies that txn.LocksAsLockUpdates propagates all the +// important fields from the txn to each intent. +func TestTxnLocksAsLockUpdates(t *testing.T) { defer leaktest.AfterTest(t)() ts := hlc.Timestamp{WallTime: 1} @@ -1912,9 +1912,8 @@ func TestAsLockUpdates(t *testing.T) { txn.Status = COMMITTED txn.IgnoredSeqNums = []enginepb.IgnoredSeqNumRange{{Start: 0, End: 0}} - - spans := []Span{{Key: Key("a"), EndKey: Key("b")}} - for _, intent := range AsLockUpdates(&txn, spans) { + txn.LockSpans = []Span{{Key: Key("a"), EndKey: Key("b")}} + for _, intent := range txn.LocksAsLockUpdates() { require.Equal(t, txn.Status, intent.Status) require.Equal(t, txn.IgnoredSeqNums, intent.IgnoredSeqNums) require.Equal(t, txn.TxnMeta, intent.Txn) diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index 38a6e568eea9..c0c98ef6f3cb 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -979,6 +979,7 @@ func benchmarkAggregateFunction( aggInputTypes []*types.T, groupSize int, nullProb float64, + numInputBatches int, ) { rng, _ := randutil.NewPseudoRand() ctx := context.Background() @@ -987,7 +988,6 @@ func benchmarkAggregateFunction( aggMemAcc := evalCtx.Mon.MakeBoundAccount() defer aggMemAcc.Close(ctx) evalCtx.SingleDatumAggMemAccount = &aggMemAcc - const numInputBatches = 64 const bytesFixedLength = 8 typs := append([]*types.T{types.Int}, aggInputTypes...) nTuples := numInputBatches * coldata.BatchSize() @@ -996,12 +996,19 @@ func benchmarkAggregateFunction( cols[i] = testAllocator.NewMemColumn(typs[i], nTuples) } groups := cols[0].Int64() - curGroup := -1 - for i := 0; i < nTuples; i++ { - if groupSize == 1 || i%groupSize == 0 { - curGroup++ + if agg.name == "hash" { + numGroups := nTuples / groupSize + for i := 0; i < nTuples; i++ { + groups[i] = int64(rng.Intn(numGroups)) + } + } else { + curGroup := -1 + for i := 0; i < nTuples; i++ { + if groupSize == 1 || i%groupSize == 0 { + curGroup++ + } + groups[i] = int64(curGroup) } - groups[i] = int64(curGroup) } for _, col := range cols[1:] { coldatatestutils.RandomVec(coldatatestutils.RandomVecArgs{ @@ -1064,11 +1071,21 @@ func benchmarkAggregateFunction( // For COUNT_ROWS we'll just use 8 bytes. argumentsSize = 8 } + var inputTypesString string + switch len(aggInputTypes) { + case 1: + // Override the string so that the name of the benchmark was the same + // as in pre-20.2 releases (which allows us to compare against old + // numbers). + inputTypesString = aggInputTypes[0].String() + default: + inputTypesString = fmt.Sprintf("%s", aggInputTypes) + } b.Run(fmt.Sprintf( "%s/%s/%s/groupSize=%d/hasNulls=%t/numInputBatches=%d", - fName, agg.name, aggInputTypes, groupSize, nullProb > 0, numInputBatches), + fName, agg.name, inputTypesString, groupSize, nullProb > 0, numInputBatches), func(b *testing.B) { - b.SetBytes(int64(argumentsSize * len(aggInputTypes) * nTuples)) + b.SetBytes(int64(argumentsSize * nTuples)) for i := 0; i < b.N; i++ { a.(resetter).reset(ctx) // Exhaust aggregator until all batches have been read. @@ -1086,10 +1103,20 @@ func benchmarkAggregateFunction( // depending on the parameters of the input. func BenchmarkAggregator(b *testing.B) { aggFn := execinfrapb.AggregatorSpec_MIN + numBatches := []int{4, 64, 1024} + groupSizes := []int{1, 2, 32, 128, coldata.BatchSize() / 2, coldata.BatchSize()} + if testing.Short() { + numBatches = []int{64} + groupSizes = []int{1, coldata.BatchSize()} + } for _, agg := range aggTypes { - for _, groupSize := range []int{1, 2, 32, 128, coldata.BatchSize() / 2, coldata.BatchSize()} { - for _, nullProb := range []float64{0.0, nullProbability} { - benchmarkAggregateFunction(b, agg, aggFn, []*types.T{types.Int}, groupSize, nullProb) + for _, numInputBatches := range numBatches { + for _, groupSize := range groupSizes { + for _, nullProb := range []float64{0.0, nullProbability} { + benchmarkAggregateFunction( + b, agg, aggFn, []*types.T{types.Int}, groupSize, nullProb, numInputBatches, + ) + } } } } @@ -1101,7 +1128,12 @@ func BenchmarkAggregator(b *testing.B) { // enough signal on the speeds of aggregate functions. For more diverse // configurations look at BenchmarkAggregator. func BenchmarkAllOptimizedAggregateFunctions(b *testing.B) { - for aggFnNumber := 0; aggFnNumber < len(execinfrapb.AggregatorSpec_Func_name); aggFnNumber++ { + const numInputBatches = 64 + numFnsToRun := len(execinfrapb.AggregatorSpec_Func_name) + if testing.Short() { + numFnsToRun = 1 + } + for aggFnNumber := 0; aggFnNumber < numFnsToRun; aggFnNumber++ { aggFn := execinfrapb.AggregatorSpec_Func(aggFnNumber) if !isAggOptimized(aggFn) { continue @@ -1118,7 +1150,9 @@ func BenchmarkAllOptimizedAggregateFunctions(b *testing.B) { aggInputTypes = []*types.T{types.Int} } for _, groupSize := range []int{1, coldata.BatchSize()} { - benchmarkAggregateFunction(b, agg, aggFn, aggInputTypes, groupSize, nullProbability) + benchmarkAggregateFunction( + b, agg, aggFn, aggInputTypes, groupSize, nullProbability, numInputBatches, + ) } } } @@ -1269,9 +1303,9 @@ func TestHashAggregator(t *testing.T) { input: tuples{ {0, 3}, {0, 4}, - {HashTableNumBuckets, 6}, + {coldata.BatchSize(), 6}, {0, 5}, - {HashTableNumBuckets, 7}, + {coldata.BatchSize(), 7}, }, typs: []*types.T{types.Int, types.Int}, groupCols: []uint32{0}, @@ -1333,25 +1367,20 @@ func TestHashAggregator(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) defer evalCtx.Stop(context.Background()) - for _, numOfHashBuckets := range []int{0 /* no limit */, 1, coldata.BatchSize()} { - for _, tc := range tcs { - if err := tc.init(); err != nil { - t.Fatal(err) - } - constructors, constArguments, outputTypes, err := ProcessAggregations( - &evalCtx, nil /* semaCtx */, tc.spec.Aggregations, tc.typs, - ) - require.NoError(t, err) - log.Infof(context.Background(), "numOfHashBuckets=%d", numOfHashBuckets) - runTests(t, []tuples{tc.input}, tc.expected, unorderedVerifier, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { - a, err := NewHashAggregator( - testAllocator, testMemAcc, sources[0], tc.typs, tc.spec, - &evalCtx, constructors, constArguments, outputTypes, - ) - a.(*hashAggregator).testingKnobs.numOfHashBuckets = uint64(numOfHashBuckets) - return a, err - }) + for _, tc := range tcs { + if err := tc.init(); err != nil { + t.Fatal(err) } + constructors, constArguments, outputTypes, err := ProcessAggregations( + &evalCtx, nil /* semaCtx */, tc.spec.Aggregations, tc.typs, + ) + require.NoError(t, err) + runTests(t, []tuples{tc.input}, tc.expected, unorderedVerifier, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { + return NewHashAggregator( + testAllocator, testMemAcc, sources[0], tc.typs, tc.spec, + &evalCtx, constructors, constArguments, outputTypes, + ) + }) } } diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index f1b66d66bcfa..85467e7d8c71 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -742,7 +742,7 @@ func NewColOperator( // input is about 0.01 or less. result.Op = colexec.NewUnorderedDistinct( colmem.NewAllocator(ctx, distinctMemAccount, factory), inputs[0], - core.Distinct.DistinctColumns, result.ColumnTypes, colexec.HashTableNumBuckets, + core.Distinct.DistinctColumns, result.ColumnTypes, ) } diff --git a/pkg/sql/colexec/default_agg_test.go b/pkg/sql/colexec/default_agg_test.go index 1098c827841e..fd38e0ce5e43 100644 --- a/pkg/sql/colexec/default_agg_test.go +++ b/pkg/sql/colexec/default_agg_test.go @@ -157,11 +157,14 @@ func TestDefaultAggregateFunc(t *testing.T) { } func BenchmarkDefaultAggregateFunction(b *testing.B) { + const numInputBatches = 64 aggFn := execinfrapb.AggregatorSpec_STRING_AGG for _, agg := range aggTypes { for _, groupSize := range []int{1, 2, 32, 128, coldata.BatchSize() / 2, coldata.BatchSize()} { for _, nullProb := range []float64{0.0, nullProbability} { - benchmarkAggregateFunction(b, agg, aggFn, []*types.T{types.String, types.String}, groupSize, nullProb) + benchmarkAggregateFunction( + b, agg, aggFn, []*types.T{types.String, types.String}, groupSize, nullProb, numInputBatches, + ) } } } diff --git a/pkg/sql/colexec/distinct_test.go b/pkg/sql/colexec/distinct_test.go index 4799e19ed6fe..3eac176a8794 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -185,15 +185,13 @@ func TestDistinct(t *testing.T) { } for _, tc := range tcs { - for _, numOfBuckets := range []uint64{1, 3, 5, HashTableNumBuckets} { - log.Infof(context.Background(), "unordered/numOfBuckets=%d", numOfBuckets) - runTestsWithTyps(t, []tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, orderedVerifier, - func(input []colexecbase.Operator) (colexecbase.Operator, error) { - return NewUnorderedDistinct( - testAllocator, input[0], tc.distinctCols, tc.typs, - numOfBuckets), nil - }) - } + log.Infof(context.Background(), "unordered") + runTestsWithTyps(t, []tuples{tc.tuples}, [][]*types.T{tc.typs}, tc.expected, orderedVerifier, + func(input []colexecbase.Operator) (colexecbase.Operator, error) { + return NewUnorderedDistinct( + testAllocator, input[0], tc.distinctCols, tc.typs, + ), nil + }) if tc.isOrderedOnDistinctCols { for numOrderedCols := 1; numOrderedCols < len(tc.distinctCols); numOrderedCols++ { log.Infof(context.Background(), "partiallyOrdered/ordCols=%d", numOrderedCols) @@ -224,7 +222,7 @@ func BenchmarkDistinct(b *testing.B) { distinctConstructors := []func(*colmem.Allocator, colexecbase.Operator, []uint32, int, []*types.T) (colexecbase.Operator, error){ func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { - return NewUnorderedDistinct(allocator, input, distinctCols, typs, HashTableNumBuckets), nil + return NewUnorderedDistinct(allocator, input, distinctCols, typs), nil }, func(allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecbase.Operator, error) { return newPartiallyOrderedDistinct(allocator, input, distinctCols, distinctCols[:numOrderedCols], typs) diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index e2843a4f4b1c..6c9568635700 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -429,16 +429,14 @@ func (hj *externalHashJoiner) partitionBatch( return } scratchBatch := hj.scratch.leftBatch - sourceSpec := hj.spec.left + eqCols := hj.spec.left.eqCols partitioner := hj.leftPartitioner if side == rightSide { scratchBatch = hj.scratch.rightBatch - sourceSpec = hj.spec.right + eqCols = hj.spec.right.eqCols partitioner = hj.rightPartitioner } - selections := hj.tupleDistributor.distribute( - ctx, batch, sourceSpec.sourceTypes, sourceSpec.eqCols, - ) + selections := hj.tupleDistributor.distribute(ctx, batch, eqCols) for idx, sel := range selections { partitionIdx := hj.partitionIdxOffset + idx if len(sel) > 0 { diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index caf6bd42f92f..f365319435d7 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -91,13 +91,6 @@ type hashAggregator struct { output coldata.Batch - testingKnobs struct { - // numOfHashBuckets is the number of hash buckets that each tuple will be - // assigned to. When it is 0, hash aggregator will not enforce maximum - // number of hash buckets. It is used to test hash collision. - numOfHashBuckets uint64 - } - aggFnsAlloc *aggregateFuncsAlloc hashAlloc hashAggBucketAlloc datumAlloc sqlbase.DatumAlloc @@ -160,14 +153,12 @@ func (op *hashAggregator) Init() { op.scratch.eqChains = make([][]int, coldata.BatchSize()) op.scratch.intSlice = make([]int, coldata.BatchSize()) op.scratch.anotherIntSlice = make([]int, coldata.BatchSize()) - // TODO(yuzefovich): tune this. - numOfHashBuckets := uint64(HashTableNumBuckets) - if op.testingKnobs.numOfHashBuckets != 0 { - numOfHashBuckets = op.testingKnobs.numOfHashBuckets - } + // This number was chosen after running the micro-benchmarks and relevant + // TPCH queries using tpchvec/bench. + const hashTableLoadFactor = 0.25 op.ht = newHashTable( op.allocator, - numOfHashBuckets, + hashTableLoadFactor, op.inputTypes, op.spec.GroupCols, true, /* allowNullEquality */ diff --git a/pkg/sql/colexec/hash_utils.go b/pkg/sql/colexec/hash_utils.go index 8946eda8ed72..c40889159460 100644 --- a/pkg/sql/colexec/hash_utils.go +++ b/pkg/sql/colexec/hash_utils.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/sql/types" ) // initHash, rehash, and finalizeHash work together to compute the hash value @@ -126,7 +125,7 @@ func newTupleHashDistributor(initHashValue uint64, numOutputs int) *tupleHashDis } func (d *tupleHashDistributor) distribute( - ctx context.Context, b coldata.Batch, types []*types.T, hashCols []uint32, + ctx context.Context, b coldata.Batch, hashCols []uint32, ) [][]int { n := b.Length() initHash(d.buckets, n, d.initHashValue) diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index dfe336644b18..99c2a44ce4f2 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -226,9 +226,12 @@ func (hj *hashJoiner) Init() { allowNullEquality = true probeMode = hashTableDeletingProbeMode } + // This number was chosen after running the micro-benchmarks and relevant + // TPCH queries using tpchvec/bench. + const hashTableLoadFactor = 8.0 hj.ht = newHashTable( hj.allocator, - HashTableNumBuckets, + hashTableLoadFactor, hj.spec.right.sourceTypes, hj.spec.right.eqCols, allowNullEquality, diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index 096be9673298..5a0de5f5fe9d 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -380,22 +380,22 @@ func init() { leftTuples: tuples{ {0}, - {HashTableNumBuckets}, - {HashTableNumBuckets}, - {HashTableNumBuckets}, + {coldata.BatchSize()}, + {coldata.BatchSize()}, + {coldata.BatchSize()}, {0}, - {HashTableNumBuckets * 2}, + {coldata.BatchSize() * 2}, {1}, {1}, - {HashTableNumBuckets + 1}, + {coldata.BatchSize() + 1}, }, rightTuples: tuples{ - {HashTableNumBuckets}, - {HashTableNumBuckets * 2}, - {HashTableNumBuckets * 3}, + {coldata.BatchSize()}, + {coldata.BatchSize() * 2}, + {coldata.BatchSize() * 3}, {0}, {1}, - {HashTableNumBuckets + 1}, + {coldata.BatchSize() + 1}, }, leftEqCols: []uint32{0}, @@ -408,15 +408,15 @@ func init() { rightEqColsAreKey: false, expected: tuples{ - {HashTableNumBuckets, HashTableNumBuckets}, - {HashTableNumBuckets, HashTableNumBuckets}, - {HashTableNumBuckets, HashTableNumBuckets}, - {HashTableNumBuckets * 2, HashTableNumBuckets * 2}, + {coldata.BatchSize(), coldata.BatchSize()}, + {coldata.BatchSize(), coldata.BatchSize()}, + {coldata.BatchSize(), coldata.BatchSize()}, + {coldata.BatchSize() * 2, coldata.BatchSize() * 2}, {0, 0}, {0, 0}, {1, 1}, {1, 1}, - {HashTableNumBuckets + 1, HashTableNumBuckets + 1}, + {coldata.BatchSize() + 1, coldata.BatchSize() + 1}, }, }, { @@ -501,14 +501,14 @@ func init() { // hash to the same bucket. leftTuples: tuples{ {0}, - {HashTableNumBuckets}, - {HashTableNumBuckets * 2}, - {HashTableNumBuckets * 3}, + {coldata.BatchSize()}, + {coldata.BatchSize() * 2}, + {coldata.BatchSize() * 3}, }, rightTuples: tuples{ {0}, - {HashTableNumBuckets}, - {HashTableNumBuckets * 3}, + {coldata.BatchSize()}, + {coldata.BatchSize() * 3}, }, leftEqCols: []uint32{0}, @@ -521,8 +521,8 @@ func init() { expected: tuples{ {0}, - {HashTableNumBuckets}, - {HashTableNumBuckets * 3}, + {coldata.BatchSize()}, + {coldata.BatchSize() * 3}, }, }, { @@ -607,17 +607,17 @@ func init() { // Test multiple column with values that hash to the same bucket. leftTuples: tuples{ {10, 0, 0}, - {20, 0, HashTableNumBuckets}, - {40, HashTableNumBuckets, 0}, - {50, HashTableNumBuckets, HashTableNumBuckets}, - {60, HashTableNumBuckets * 2, 0}, - {70, HashTableNumBuckets * 2, HashTableNumBuckets}, + {20, 0, coldata.BatchSize()}, + {40, coldata.BatchSize(), 0}, + {50, coldata.BatchSize(), coldata.BatchSize()}, + {60, coldata.BatchSize() * 2, 0}, + {70, coldata.BatchSize() * 2, coldata.BatchSize()}, }, rightTuples: tuples{ - {0, HashTableNumBuckets}, - {HashTableNumBuckets * 2, HashTableNumBuckets}, + {0, coldata.BatchSize()}, + {coldata.BatchSize() * 2, coldata.BatchSize()}, {0, 0}, - {0, HashTableNumBuckets * 2}, + {0, coldata.BatchSize() * 2}, }, leftEqCols: []uint32{1, 2}, @@ -629,8 +629,8 @@ func init() { rightEqColsAreKey: true, expected: tuples{ - {20, 0, HashTableNumBuckets}, - {70, HashTableNumBuckets * 2, HashTableNumBuckets}, + {20, 0, coldata.BatchSize()}, + {70, coldata.BatchSize() * 2, coldata.BatchSize()}, {10, 0, 0}, }, }, diff --git a/pkg/sql/colexec/hashtable.go b/pkg/sql/colexec/hashtable.go index b259e347fef0..6d9da4b55664 100644 --- a/pkg/sql/colexec/hashtable.go +++ b/pkg/sql/colexec/hashtable.go @@ -13,6 +13,7 @@ package colexec import ( "context" "fmt" + "unsafe" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" @@ -22,10 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// HashTableNumBuckets is the default number of buckets in the colexec hashtable. -// TODO(yuzefovich): support rehashing instead of large fixed bucket size. -const HashTableNumBuckets = 1 << 16 - // hashTableBuildMode represents different modes in which the hashTable can be // built. type hashTableBuildMode int @@ -133,6 +130,18 @@ type hashTableProbeBuffer struct { type hashTable struct { allocator *colmem.Allocator + // internalMemHelper is a utility struct that helps with memory accounting + // of the internal memory of the hash table (auxiliary slices). + internalMemHelper struct { + // constSlicesAreAccountedFor indicates whether we have already + // accounted for the memory used by the slices that are constant in + // size. + constSlicesAreAccountedFor bool + // dynamicSlicesNumUint64AccountedFor stores the number of uint64 from + // the dynamic slices that we have already accounted for. + dynamicSlicesNumUint64AccountedFor int64 + } + // buildScratch contains the scratch buffers required for the build table. buildScratch hashTableBuildBuffer @@ -162,6 +171,9 @@ type hashTable struct { // numBuckets returns the number of buckets the hashTable employs. This is // equivalent to the size of first. numBuckets uint64 + // loadFactor determines the average number of tuples per bucket exceeding + // of which will trigger resizing the hash table. + loadFactor float64 // allowNullEquality determines if NULL keys should be treated as equal to // each other. @@ -177,9 +189,16 @@ type hashTable struct { var _ resetter = &hashTable{} +// newHashTable returns a new hashTable. +// - loadFactor determines the average number of tuples per bucket which, if +// exceeded, will trigger resizing the hash table. This number can have a +// noticeable effect on the performance, so every user of the hash table should +// choose the number that works well for the corresponding use case. 1.0 could +// be used as the initial default value, and most likely the best value will be +// in [0.1, 10.0] range. func newHashTable( allocator *colmem.Allocator, - numBuckets uint64, + loadFactor float64, sourceTypes []*types.T, eqCols []uint32, allowNullEquality bool, @@ -191,11 +210,28 @@ func newHashTable( // assert that it is not requested. colexecerror.InternalError("hashTableDeletingProbeMode is supported only when null equality is allowed") } + // This number was chosen after running benchmarks of all users of the hash + // table (hash joiner, hash aggregator, unordered distinct). The reasoning + // for why using coldata.BatchSize() as the initial number of buckets make + // sense: + // - on one hand, we make several other allocations that have to be at + // least coldata.BatchSize() in size, so we don't win much in the case of + // the input with small number of tuples; + // - on the other hand, if we start out with a larger number, we won't be + // using the vast of majority of the buckets on the input with small number + // of tuples (a downside) while not gaining much in the case of the input + // with large number of tuples. + initialNumHashBuckets := uint64(coldata.BatchSize()) + // Note that we don't perform memory accounting of the internal memory here + // and delay it till buildFromBufferedTuples in order to appease *-disk + // logic test configs (our disk-spilling infrastructure doesn't know how to + // fallback to disk when a memory limit is hit in the constructor methods + // of the operators or in Init() implementations). ht := &hashTable{ allocator: allocator, buildScratch: hashTableBuildBuffer{ - first: make([]uint64, numBuckets), + first: make([]uint64, initialNumHashBuckets), }, probeScratch: hashTableProbeBuffer{ @@ -209,14 +245,15 @@ func newHashTable( vals: newAppendOnlyBufferedBatch(allocator, sourceTypes), keyCols: eqCols, - numBuckets: numBuckets, + numBuckets: initialNumHashBuckets, + loadFactor: loadFactor, allowNullEquality: allowNullEquality, buildMode: buildMode, probeMode: probeMode, } if buildMode == hashTableDistinctBuildMode { - ht.probeScratch.first = make([]uint64, numBuckets) + ht.probeScratch.first = make([]uint64, initialNumHashBuckets) ht.probeScratch.next = make([]uint64, coldata.BatchSize()+1) ht.buildScratch.next = make([]uint64, 1, coldata.BatchSize()+1) ht.probeScratch.hashBuffer = make([]uint64, coldata.BatchSize()) @@ -226,51 +263,96 @@ func newHashTable( return ht } +// shouldResize returns whether the hash table storing numTuples should be +// resized in order to not exceed the load factor given the current number of +// buckets. +func (ht *hashTable) shouldResize(numTuples int) bool { + return float64(numTuples)/float64(ht.numBuckets) > ht.loadFactor +} + +const sizeOfUint64 = int64(unsafe.Sizeof(uint64(0))) + +// accountForConstSlices checks whether we have already accounted for the +// memory used by the slices that are constant in size and adjusts the +// allocator accordingly if we haven't. +func (ht *hashTable) accountForConstSlices() { + if ht.internalMemHelper.constSlicesAreAccountedFor { + return + } + const sizeOfBool = int64(unsafe.Sizeof(true)) + internalMemUsed := sizeOfUint64 * int64(cap(ht.probeScratch.buckets)+ + cap(ht.probeScratch.groupID)+cap(ht.probeScratch.headID)+cap(ht.probeScratch.toCheck)) + internalMemUsed += sizeOfBool * int64(cap(ht.probeScratch.differs)+cap(ht.probeScratch.distinct)) + ht.allocator.AdjustMemoryUsage(internalMemUsed) + ht.internalMemHelper.constSlicesAreAccountedFor = true +} + +// buildFromBufferedTuples builds the hash table from already buffered tuples +// in ht.vals. It'll determine the appropriate number of buckets that satisfy +// the target load factor. +func (ht *hashTable) buildFromBufferedTuples(ctx context.Context) { + for ht.shouldResize(ht.vals.Length()) { + ht.numBuckets *= 2 + } + ht.buildScratch.first = maybeAllocateUint64Array(ht.buildScratch.first, int(ht.numBuckets)) + if ht.probeScratch.first != nil { + ht.probeScratch.first = maybeAllocateUint64Array(ht.probeScratch.first, int(ht.numBuckets)) + } + keyCols := make([]coldata.Vec, len(ht.keyCols)) + for i, colIdx := range ht.keyCols { + keyCols[i] = ht.vals.ColVec(int(colIdx)) + } + // ht.next is used to store the computed hash value of each key. + ht.buildScratch.next = maybeAllocateUint64Array(ht.buildScratch.next, ht.vals.Length()+1) + ht.computeBuckets(ctx, ht.buildScratch.next[1:], keyCols, ht.vals.Length(), nil) + ht.buildNextChains(ctx, ht.buildScratch.first, ht.buildScratch.next, 1, uint64(ht.vals.Length())) + // Account for memory used by the internal auxiliary slices. + ht.accountForConstSlices() + // Note that if ht.probeScratch.first is nil, it'll have zero capacity. + newUint64Count := int64(cap(ht.buildScratch.first) + cap(ht.probeScratch.first) + cap(ht.buildScratch.next)) + ht.allocator.AdjustMemoryUsage(sizeOfUint64 * (newUint64Count - ht.internalMemHelper.dynamicSlicesNumUint64AccountedFor)) + ht.internalMemHelper.dynamicSlicesNumUint64AccountedFor = newUint64Count +} + // build executes the entirety of the hash table build phase using the input // as the build source. The input is entirely consumed in the process. func (ht *hashTable) build(ctx context.Context, input colexecbase.Operator) { - nKeyCols := len(ht.keyCols) - switch ht.buildMode { case hashTableFullBuildMode: + // We're using the hash table with the full build mode in which we will + // fully buffer all tuples from the input first and only then we'll + // build the hash table. Such approach allows us to compute the desired + // number of hash buckets for the target load factor (this is done in + // buildFromBufferedTuples()). for { batch := input.Next(ctx) if batch.Length() == 0 { break } - ht.allocator.PerformOperation(ht.vals.ColVecs(), func() { ht.vals.append(batch, 0 /* startIdx */, batch.Length()) }) } + ht.buildFromBufferedTuples(ctx) - keyCols := make([]coldata.Vec, nKeyCols) - for i := 0; i < nKeyCols; i++ { - keyCols[i] = ht.vals.ColVec(int(ht.keyCols[i])) - } - - // ht.next is used to store the computed hash value of each key. - ht.buildScratch.next = maybeAllocateUint64Array(ht.buildScratch.next, ht.vals.Length()+1) - ht.computeBuckets(ctx, ht.buildScratch.next[1:], keyCols, ht.vals.Length(), nil) - ht.buildNextChains(ctx, ht.buildScratch.first, ht.buildScratch.next, 1, uint64(ht.vals.Length())) case hashTableDistinctBuildMode: for { batch := input.Next(ctx) if batch.Length() == 0 { break } - ht.computeHashAndBuildChains(ctx, batch) ht.removeDuplicates(batch, ht.probeScratch.keys, ht.probeScratch.first, ht.probeScratch.next, ht.checkProbeForDistinct) - // We only check duplicates when there is at least one buffered // tuple. if ht.vals.Length() > 0 { ht.removeDuplicates(batch, ht.probeScratch.keys, ht.buildScratch.first, ht.buildScratch.next, ht.checkBuildForDistinct) } - - ht.appendAllDistinct(ctx, batch) + if batch.Length() > 0 { + ht.appendAllDistinct(ctx, batch) + } } + default: colexecerror.InternalError(fmt.Sprintf("hashTable in unhandled state")) } @@ -286,20 +368,26 @@ func (ht *hashTable) computeHashAndBuildChains(ctx context.Context, batch coldat ht.probeScratch.keys[i] = srcVecs[keyCol] } - ht.computeBuckets(ctx, ht.probeScratch.next[1:], ht.probeScratch.keys, batch.Length(), batch.Selection()) + batchLength := batch.Length() + ht.computeBuckets(ctx, ht.probeScratch.next[1:], ht.probeScratch.keys, batchLength, batch.Selection()) copy(ht.probeScratch.hashBuffer, ht.probeScratch.next[1:]) - // We should not zero out the entire `first` buffer here since the size of - // the `first` buffer same as the hash range (2^16) by default. The size - // of the hashBuffer is same as the batch size which is often a lot - // smaller than the hash range. Since we are only concerned with tuples - // inside the hashBuffer, we only need to zero out the corresponding - // entries in the `first` buffer that occurred in the hashBuffer. - for _, hash := range ht.probeScratch.hashBuffer[:batch.Length()] { - ht.probeScratch.first[hash] = 0 + // We need to zero out 'first' buffer for all hash codes present in + // hashBuffer, and there are two possible approaches that we choose from + // based on a heuristic - we can either iterate over all hash codes and + // zero out only the relevant elements (beneficial when 'first' buffer is + // at least batchLength in size) or zero out the whole 'first' buffer + // (beneficial otherwise). + if batchLength < len(ht.probeScratch.first) { + for _, hash := range ht.probeScratch.hashBuffer[:batchLength] { + ht.probeScratch.first[hash] = 0 + } + } else { + for n := 0; n < len(ht.probeScratch.first); n += copy(ht.probeScratch.first[n:], zeroUint64Column) { + } } - ht.buildNextChains(ctx, ht.probeScratch.first, ht.probeScratch.next, 1 /* offset */, uint64(batch.Length())) + ht.buildNextChains(ctx, ht.probeScratch.first, ht.probeScratch.next, 1 /* offset */, uint64(batchLength)) } // findBuckets finds the buckets for all tuples in batch when probing against a @@ -355,6 +443,9 @@ func (ht *hashTable) appendAllDistinct(ctx context.Context, batch coldata.Batch) }) ht.buildScratch.next = append(ht.buildScratch.next, ht.probeScratch.hashBuffer[:batch.Length()]...) ht.buildNextChains(ctx, ht.buildScratch.first, ht.buildScratch.next, numBuffered+1, uint64(batch.Length())) + if ht.shouldResize(ht.vals.Length()) { + ht.buildFromBufferedTuples(ctx) + } } // checkCols performs a column by column checkCol on the key columns. @@ -394,17 +485,20 @@ func (ht *hashTable) checkColsForDistinctTuples( func (ht *hashTable) computeBuckets( ctx context.Context, buckets []uint64, keys []coldata.Vec, nKeys int, sel []int, ) { - initHash(buckets, nKeys, defaultInitHashValue) - if nKeys == 0 { // No work to do - avoid doing the loops below. return } - // Check if we received a batch with more tuples than the current - // allocation size and increase it if so. - if nKeys > ht.datumAlloc.AllocSize { + initHash(buckets, nKeys, defaultInitHashValue) + + // Check if we received more tuples than the current allocation size and + // increase it if so (limiting it by coldata.BatchSize()). + if nKeys > ht.datumAlloc.AllocSize && ht.datumAlloc.AllocSize < coldata.BatchSize() { ht.datumAlloc.AllocSize = nKeys + if ht.datumAlloc.AllocSize > coldata.BatchSize() { + ht.datumAlloc.AllocSize = coldata.BatchSize() + } } for i := range ht.keyCols { @@ -577,11 +671,11 @@ func (ht *hashTable) reset(_ context.Context) { // they are used (these slices are not used in all of the code paths). // ht.probeScratch.buckets doesn't need to be reset because buckets are // always initialized when computing the hash. - copy(ht.probeScratch.groupID[:coldata.BatchSize()], zeroUint64Column) + copy(ht.probeScratch.groupID, zeroUint64Column) // ht.probeScratch.toCheck doesn't need to be reset because it is populated // manually every time before checking the columns. - copy(ht.probeScratch.headID[:coldata.BatchSize()], zeroUint64Column) - copy(ht.probeScratch.differs[:coldata.BatchSize()], zeroBoolColumn) + copy(ht.probeScratch.headID, zeroUint64Column) + copy(ht.probeScratch.differs, zeroBoolColumn) copy(ht.probeScratch.distinct, zeroBoolColumn) if ht.buildMode == hashTableDistinctBuildMode && cap(ht.buildScratch.next) > 0 { // In "distinct" build mode, ht.buildScratch.next is populated diff --git a/pkg/sql/colexec/hashtable_distinct.eg.go b/pkg/sql/colexec/hashtable_distinct.eg.go index 9cddb264ee59..676be9d629e6 100644 --- a/pkg/sql/colexec/hashtable_distinct.eg.go +++ b/pkg/sql/colexec/hashtable_distinct.eg.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -25,9 +24,6 @@ import ( // against itself. func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel []int) { probeVec, buildVec, probeSel := vec, vec, sel - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `overloadHelper`. - _overloadHelper := ht.overloadHelper switch probeVec.CanonicalTypeFamily() { case types.BoolFamily: switch probeVec.Type().Width() { @@ -952,11 +948,12 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel case -1: default: switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: + case types.DecimalFamily: switch buildVec.Type().Width() { - case 16: + case -1: + default: probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int16() + buildKeys := buildVec.Decimal() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1003,13 +1000,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1047,13 +1038,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1091,13 +1076,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1136,13 +1115,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1178,13 +1151,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1238,13 +1205,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1280,13 +1241,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1322,13 +1277,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1365,13 +1314,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1405,13 +1348,7 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1422,9 +1359,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - case 32: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int32() + } + } + } + case types.IntFamily: + switch probeVec.Type().Width() { + case 16: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: + switch buildVec.Type().Width() { + case 16: + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1473,9 +1419,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1517,9 +1468,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1561,9 +1517,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1606,9 +1567,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1648,9 +1614,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1708,9 +1679,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1750,9 +1726,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1792,9 +1773,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1835,9 +1821,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1875,9 +1866,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1890,10 +1886,9 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - case -1: - default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int64() + case 32: + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1942,9 +1937,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1986,9 +1986,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2030,9 +2035,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2075,9 +2085,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2117,9 +2132,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2177,9 +2197,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2219,9 +2244,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2261,9 +2291,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2304,9 +2339,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2344,9 +2384,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2359,13 +2404,10 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - } - case types.FloatFamily: - switch buildVec.Type().Width() { case -1: default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Float64() + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -2414,11 +2456,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2460,13 +2505,16 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2506,11 +2554,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2553,11 +2604,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2597,11 +2651,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2659,11 +2716,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2703,11 +2763,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2747,11 +2810,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2792,11 +2858,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2834,11 +2903,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2852,12 +2924,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - case types.DecimalFamily: + } + case 32: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Decimal() + case 16: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -2904,7 +2978,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2942,7 +3027,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2980,7 +3076,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3019,7 +3126,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3055,7 +3173,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3109,7 +3238,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3145,7 +3285,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3181,7 +3332,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3218,7 +3380,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3252,7 +3425,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3263,18 +3447,9 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - } - } - } - case types.IntFamily: - switch probeVec.Type().Width() { - case 16: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int16() + case 32: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -3790,9 +3965,10 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - case 32: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int32() + case -1: + default: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4308,10 +4484,16 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - case -1: - default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int64() + } + } + case -1: + default: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: + switch buildVec.Type().Width() { + case 16: + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4827,13 +5009,9 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Float64() + case 32: + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4882,21 +5060,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -4939,21 +5109,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -4996,21 +5158,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5054,21 +5208,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5109,21 +5255,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5182,21 +5320,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5237,21 +5367,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5292,21 +5414,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5348,21 +5462,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5401,21 +5507,13 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5429,13 +5527,10 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { case -1: default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Decimal() + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -5484,9 +5579,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5528,9 +5628,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5572,9 +5677,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5617,9 +5727,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5659,9 +5774,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5719,9 +5839,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5761,9 +5886,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5803,9 +5933,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5846,9 +5981,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5886,9 +6026,14 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5903,13 +6048,18 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel } } } - case 32: + } + case types.FloatFamily: + switch probeVec.Type().Width() { + case -1: + default: switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: + case types.FloatFamily: switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int16() + case -1: + default: + probeKeys := probeVec.Float64() + buildKeys := buildVec.Float64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -5958,13 +6108,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6007,13 +6165,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6056,13 +6222,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6106,19 +6280,27 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + unique = cmpResult != 0 + } + ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique } } @@ -6153,13 +6335,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6218,13 +6408,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6265,13 +6463,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6312,13 +6518,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6360,13 +6574,21 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6405,7660 +6627,22 @@ func (ht *hashTable) checkColAgainstItself(vec coldata.Vec, nToCheck uint64, sel var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { + } else if math.IsNaN(a) { + if math.IsNaN(b) { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 } else { - cmpResult = 0 + cmpResult = -1 } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - } - case -1: - default: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int16() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - } - } - case types.FloatFamily: - switch probeVec.Type().Width() { - case -1: - default: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int16() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - // The vector is probed against itself, so buildVec has the same - // selection vector as probeVec. - buildIdx = probeSel[keyID-1] - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) } unique = cmpResult != 0 diff --git a/pkg/sql/colexec/hashtable_full_default.eg.go b/pkg/sql/colexec/hashtable_full_default.eg.go index 4f3877b6631b..49ac673c8127 100644 --- a/pkg/sql/colexec/hashtable_full_default.eg.go +++ b/pkg/sql/colexec/hashtable_full_default.eg.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" - "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" ) @@ -29,9 +28,6 @@ import ( func (ht *hashTable) checkCol( probeVec, buildVec coldata.Vec, keyColIdx int, nToCheck uint64, probeSel []int, ) { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `overloadHelper`. - _overloadHelper := ht.overloadHelper switch probeVec.CanonicalTypeFamily() { case types.BoolFamily: switch probeVec.Type().Width() { @@ -936,11 +932,12 @@ func (ht *hashTable) checkCol( case -1: default: switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: + case types.DecimalFamily: switch buildVec.Type().Width() { - case 16: + case -1: + default: probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int16() + buildKeys := buildVec.Decimal() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -985,13 +982,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1027,13 +1018,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1069,13 +1054,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1112,13 +1091,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1152,13 +1125,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1212,13 +1179,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1254,13 +1215,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1296,13 +1251,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1339,13 +1288,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1379,13 +1322,7 @@ func (ht *hashTable) checkCol( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1396,9 +1333,18 @@ func (ht *hashTable) checkCol( } } } - case 32: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int32() + } + } + } + case types.IntFamily: + switch probeVec.Type().Width() { + case 16: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: + switch buildVec.Type().Width() { + case 16: + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1445,9 +1391,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1487,9 +1438,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1529,9 +1485,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1572,9 +1533,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1612,9 +1578,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1672,9 +1643,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1714,9 +1690,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1756,9 +1737,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1799,9 +1785,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1839,9 +1830,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1854,10 +1850,9 @@ func (ht *hashTable) checkCol( } } } - case -1: - default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int64() + case 32: + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1904,9 +1899,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1946,9 +1946,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1988,9 +1993,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2031,9 +2041,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2071,9 +2086,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2131,9 +2151,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2173,9 +2198,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2215,9 +2245,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2258,9 +2293,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2298,9 +2338,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2313,13 +2358,10 @@ func (ht *hashTable) checkCol( } } } - } - case types.FloatFamily: - switch buildVec.Type().Width() { case -1: default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Float64() + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -2366,11 +2408,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2410,13 +2455,16 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2454,11 +2502,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2499,11 +2550,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2541,11 +2595,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2603,11 +2660,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2647,11 +2707,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2691,11 +2754,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2736,11 +2802,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2778,11 +2847,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2796,12 +2868,14 @@ func (ht *hashTable) checkCol( } } } - case types.DecimalFamily: + } + case 32: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Decimal() + case 16: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -2846,7 +2920,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2882,7 +2967,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2918,7 +3014,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2955,7 +3062,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2989,7 +3107,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3043,7 +3172,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3079,7 +3219,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3115,7 +3266,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3152,7 +3314,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3186,7 +3359,18 @@ func (ht *hashTable) checkCol( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3197,18 +3381,9 @@ func (ht *hashTable) checkCol( } } } - } - } - } - case types.IntFamily: - switch probeVec.Type().Width() { - case 16: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int16() + case 32: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -3714,9 +3889,10 @@ func (ht *hashTable) checkCol( } } } - case 32: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int32() + case -1: + default: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4222,10 +4398,16 @@ func (ht *hashTable) checkCol( } } } - case -1: - default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int64() + } + } + case -1: + default: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: + switch buildVec.Type().Width() { + case 16: + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4731,13 +4913,9 @@ func (ht *hashTable) checkCol( } } } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Float64() + case 32: + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4784,21 +4962,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -4839,21 +5009,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -4894,21 +5056,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -4950,21 +5104,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5003,21 +5149,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5076,21 +5214,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5131,21 +5261,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5186,21 +5308,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5242,21 +5356,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5295,21 +5401,13 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5323,13 +5421,10 @@ func (ht *hashTable) checkCol( } } } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { case -1: default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Decimal() + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -5376,9 +5471,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5418,9 +5518,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5460,9 +5565,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5503,9 +5613,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5543,9 +5658,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5603,9 +5723,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5645,9 +5770,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5687,9 +5817,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5730,9 +5865,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5770,9 +5910,14 @@ func (ht *hashTable) checkCol( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -5787,13 +5932,18 @@ func (ht *hashTable) checkCol( } } } - case 32: + } + case types.FloatFamily: + switch probeVec.Type().Width() { + case -1: + default: switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: + case types.FloatFamily: switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int16() + case -1: + default: + probeKeys := probeVec.Float64() + buildKeys := buildVec.Float64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -5840,13 +5990,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -5887,13 +6045,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -5934,13 +6100,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -5982,19 +6156,27 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + unique = cmpResult != 0 + } + ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique } } @@ -6027,13 +6209,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6092,13 +6282,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6139,13 +6337,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6186,13 +6392,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6234,13 +6448,21 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6279,7520 +6501,22 @@ func (ht *hashTable) checkCol( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { + } else if math.IsNaN(a) { + if math.IsNaN(b) { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 } else { - cmpResult = 0 + cmpResult = -1 } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - } - case -1: - default: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int16() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - } - } - case types.FloatFamily: - switch probeVec.Type().Width() { - case -1: - default: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int16() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) } unique = cmpResult != 0 diff --git a/pkg/sql/colexec/hashtable_full_deleting.eg.go b/pkg/sql/colexec/hashtable_full_deleting.eg.go index 849df079df17..d59d150933cc 100644 --- a/pkg/sql/colexec/hashtable_full_deleting.eg.go +++ b/pkg/sql/colexec/hashtable_full_deleting.eg.go @@ -29,9 +29,6 @@ import ( func (ht *hashTable) checkColDeleting( probeVec, buildVec coldata.Vec, keyColIdx int, nToCheck uint64, probeSel []int, ) { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `overloadHelper`. - _overloadHelper := ht.overloadHelper switch probeVec.CanonicalTypeFamily() { case types.BoolFamily: switch probeVec.Type().Width() { @@ -1056,11 +1053,12 @@ func (ht *hashTable) checkColDeleting( case -1: default: switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: + case types.DecimalFamily: switch buildVec.Type().Width() { - case 16: + case -1: + default: probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int16() + buildKeys := buildVec.Decimal() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1111,13 +1109,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1159,13 +1151,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1207,13 +1193,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1256,13 +1236,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1302,13 +1276,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1368,13 +1336,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1416,13 +1378,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1464,13 +1420,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1513,13 +1463,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1559,13 +1503,7 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + cmpResult = tree.CompareDecimals(&probeVal, &buildVal) unique = cmpResult != 0 } @@ -1576,9 +1514,18 @@ func (ht *hashTable) checkColDeleting( } } } - case 32: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int32() + } + } + } + case types.IntFamily: + switch probeVec.Type().Width() { + case 16: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: + switch buildVec.Type().Width() { + case 16: + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -1631,9 +1578,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1679,9 +1631,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1727,9 +1684,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1776,9 +1738,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1822,9 +1789,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1888,9 +1860,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1936,9 +1913,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -1984,9 +1966,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2033,9 +2020,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2079,9 +2071,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2094,10 +2091,9 @@ func (ht *hashTable) checkColDeleting( } } } - case -1: - default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Int64() + case 32: + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -2150,9 +2146,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2198,9 +2199,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2246,9 +2252,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2295,9 +2306,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2341,9 +2357,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2407,9 +2428,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2455,9 +2481,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2503,9 +2534,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2552,9 +2588,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2598,9 +2639,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(buildVal)) - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -2613,13 +2659,10 @@ func (ht *hashTable) checkColDeleting( } } } - } - case types.FloatFamily: - switch buildVec.Type().Width() { case -1: default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Float64() + probeKeys := probeVec.Int16() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -2672,11 +2715,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2722,13 +2768,16 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) - } - + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -2772,11 +2821,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2823,11 +2875,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2871,11 +2926,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2939,11 +2997,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -2989,11 +3050,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -3039,11 +3103,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -3090,11 +3157,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -3138,11 +3208,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(buildVal)); err != nil { - colexecerror.ExpectedError(err) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 } - cmpResult = tree.CompareDecimals(&probeVal, tmpDec) } unique = cmpResult != 0 @@ -3156,12 +3229,14 @@ func (ht *hashTable) checkColDeleting( } } } - case types.DecimalFamily: + } + case 32: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Decimal() - buildKeys := buildVec.Decimal() + case 16: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -3212,7 +3287,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3254,7 +3340,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3296,7 +3393,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3339,7 +3447,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3379,7 +3498,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3439,7 +3569,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3481,7 +3622,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3523,7 +3675,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3566,7 +3729,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3606,7 +3780,18 @@ func (ht *hashTable) checkColDeleting( { var cmpResult int - cmpResult = tree.CompareDecimals(&probeVal, &buildVal) + + { + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } + } + unique = cmpResult != 0 } @@ -3617,18 +3802,9 @@ func (ht *hashTable) checkColDeleting( } } } - } - } - } - case types.IntFamily: - switch probeVec.Type().Width() { - case 16: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int16() + case 32: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4194,9 +4370,10 @@ func (ht *hashTable) checkColDeleting( } } } - case 32: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int32() + case -1: + default: + probeKeys := probeVec.Int32() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -4762,10 +4939,16 @@ func (ht *hashTable) checkColDeleting( } } } - case -1: - default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Int64() + } + } + case -1: + default: + switch buildVec.CanonicalTypeFamily() { + case types.IntFamily: + switch buildVec.Type().Width() { + case 16: + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int16() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -5331,13 +5514,9 @@ func (ht *hashTable) checkColDeleting( } } } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Float64() + case 32: + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int32() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -5390,21 +5569,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5451,21 +5622,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5512,21 +5675,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5574,21 +5729,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5633,21 +5780,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5712,21 +5851,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5773,21 +5904,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5834,21 +5957,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5896,21 +6011,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5955,21 +6062,13 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := float64(probeVal), float64(buildVal) + a, b := int64(probeVal), int64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } } else { - cmpResult = 1 + cmpResult = 0 } } @@ -5983,13 +6082,10 @@ func (ht *hashTable) checkColDeleting( } } } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { case -1: default: - probeKeys := probeVec.Int16() - buildKeys := buildVec.Decimal() + probeKeys := probeVec.Int64() + buildKeys := buildVec.Int64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -6042,9 +6138,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6090,9 +6191,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6138,9 +6244,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6187,9 +6298,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6233,9 +6349,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6299,9 +6420,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6347,9 +6473,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6395,9 +6526,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6444,9 +6580,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6490,9 +6631,14 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) + a, b := int64(probeVal), int64(buildVal) + if a < b { + cmpResult = -1 + } else if a > b { + cmpResult = 1 + } else { + cmpResult = 0 + } } unique = cmpResult != 0 @@ -6507,13 +6653,18 @@ func (ht *hashTable) checkColDeleting( } } } - case 32: + } + case types.FloatFamily: + switch probeVec.Type().Width() { + case -1: + default: switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: + case types.FloatFamily: switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int16() + case -1: + default: + probeKeys := probeVec.Float64() + buildKeys := buildVec.Float64() if probeSel != nil { if probeVec.MaybeHasNulls() { if buildVec.MaybeHasNulls() { @@ -6566,13 +6717,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6619,13 +6778,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6672,13 +6839,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6726,19 +6901,27 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 + } + } + + unique = cmpResult != 0 + } + ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique } } @@ -6777,13 +6960,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6848,13 +7039,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6901,13 +7100,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -6954,13 +7161,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -7008,13 +7223,21 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 + } else if math.IsNaN(a) { + if math.IsNaN(b) { + cmpResult = 0 + } else { + cmpResult = -1 + } + } else { + cmpResult = 1 } } @@ -7059,8360 +7282,22 @@ func (ht *hashTable) checkColDeleting( var cmpResult int { - a, b := int64(probeVal), int64(buildVal) + a, b := float64(probeVal), float64(buildVal) if a < b { cmpResult = -1 } else if a > b { cmpResult = 1 - } else { + } else if a == b { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { + } else if math.IsNaN(a) { + if math.IsNaN(b) { cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 } else { - cmpResult = 0 + cmpResult = -1 } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int32() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - } - case -1: - default: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int16() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := int64(probeVal), int64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else { - cmpResult = 0 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if false { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Int64() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetInt64(int64(probeVal)) - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - } - } - case types.FloatFamily: - switch probeVec.Type().Width() { - case -1: - default: - switch buildVec.CanonicalTypeFamily() { - case types.IntFamily: - switch buildVec.Type().Width() { - case 16: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int16() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case 32: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int32() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Int64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if false { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.FloatFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Float64() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - a, b := float64(probeVal), float64(buildVal) - if a < b { - cmpResult = -1 - } else if a > b { - cmpResult = 1 - } else if a == b { - cmpResult = 0 - } else if math.IsNaN(a) { - if math.IsNaN(b) { - cmpResult = 0 - } else { - cmpResult = -1 - } - } else { - cmpResult = 1 - } - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } - } - case types.DecimalFamily: - switch buildVec.Type().Width() { - case -1: - default: - probeKeys := probeVec.Float64() - buildKeys := buildVec.Decimal() - if probeSel != nil { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = probeSel[toCheck] - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } - } else { - if probeVec.MaybeHasNulls() { - if buildVec.MaybeHasNulls() { - if ht.allowNullEquality { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull && buildIsNull { - // Both values are NULLs, and since we're allowing null equality, we - // proceed to the next value to check. - continue - } else if probeIsNull { - // Only probing value is NULL, so it is different from the build value - // (which is non-NULL). We mark it as "different" and proceed to the - // next value to check. This behavior is special in case of allowing - // null equality because we don't want to reset the groupID of the - // current probing tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - probeIsNull = probeVec.Nulls().NullAt(probeIdx) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } - } else { - if buildVec.MaybeHasNulls() { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - buildIsNull = buildVec.Nulls().NullAt(buildIdx) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) - } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) - } - - unique = cmpResult != 0 - } - - ht.probeScratch.differs[toCheck] = ht.probeScratch.differs[toCheck] || unique - } - } - } - } else { - var ( - probeIdx, buildIdx int - probeIsNull, buildIsNull bool - ) - for _, toCheck := range ht.probeScratch.toCheck[:nToCheck] { - // keyID of 0 is reserved to represent the end of the next chain. - keyID := ht.probeScratch.groupID[toCheck] - if keyID != 0 { - // the build table key (calculated using keys[keyID - 1] = key) is - // compared to the corresponding probe table to determine if a match is - // found. - if ht.visited[keyID] { - // This build tuple has already been matched, so we treat - // it as different from the probe tuple. - ht.probeScratch.differs[toCheck] = true - continue - } - - probeIdx = int(toCheck) - buildIdx = int(keyID - 1) - if probeIsNull { - ht.probeScratch.groupID[toCheck] = 0 - } else if buildIsNull { - ht.probeScratch.differs[toCheck] = true - } else { - probeVal := probeKeys.Get(probeIdx) - buildVal := buildKeys.Get(buildIdx) - var unique bool - - { - var cmpResult int - - { - tmpDec := &_overloadHelper.tmpDec1 - if _, err := tmpDec.SetFloat64(float64(probeVal)); err != nil { - colexecerror.ExpectedError(err) } - cmpResult = tree.CompareDecimals(tmpDec, &buildVal) } unique = cmpResult != 0 diff --git a/pkg/sql/colexec/hashtable_tmpl.go b/pkg/sql/colexec/hashtable_tmpl.go index f3bb47e90530..1c8de8893910 100644 --- a/pkg/sql/colexec/hashtable_tmpl.go +++ b/pkg/sql/colexec/hashtable_tmpl.go @@ -198,17 +198,24 @@ func _CHECK_COL_FUNCTION_TEMPLATE(_PROBING_AGAINST_ITSELF bool, _DELETING_PROBE_ // {{$probingAgainstItself := .ProbingAgainstItself}} // {{$deletingProbeMode := .DeletingProbeMode}} // {{with .Global}} - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `overloadHelper`. - _overloadHelper := ht.overloadHelper switch probeVec.CanonicalTypeFamily() { // {{range .LeftFamilies}} + // {{$leftFamily := .LeftCanonicalFamilyStr}} case _LEFT_CANONICAL_TYPE_FAMILY: switch probeVec.Type().Width() { // {{range .LeftWidths}} case _LEFT_TYPE_WIDTH: switch buildVec.CanonicalTypeFamily() { // {{range .RightFamilies}} + // {{$rightFamily := .RightCanonicalFamilyStr}} + // {{/* + // We currently only support the cases of same-type as well as + // integers of mixed widths in the equality conditions (all + // other allowed mixed-type comparisons are pushed into the ON + // condition, see #43060), so we will generate the code only + // for same-type comparisons and for integer ones. + // */}} + // {{if or (eq $leftFamily $rightFamily) (and (eq $leftFamily "types.IntFamily") (eq $rightFamily "types.IntFamily"))}} case _RIGHT_CANONICAL_TYPE_FAMILY: switch buildVec.Type().Width() { // {{range .RightWidths}} @@ -223,6 +230,7 @@ func _CHECK_COL_FUNCTION_TEMPLATE(_PROBING_AGAINST_ITSELF bool, _DELETING_PROBE_ // {{end}} } // {{end}} + // {{end}} } // {{end}} } diff --git a/pkg/sql/colexec/partially_ordered_distinct.go b/pkg/sql/colexec/partially_ordered_distinct.go index 408f927253b2..43adc65d096d 100644 --- a/pkg/sql/colexec/partially_ordered_distinct.go +++ b/pkg/sql/colexec/partially_ordered_distinct.go @@ -23,9 +23,6 @@ import ( "github.com/cockroachdb/errors" ) -// TODO(yuzefovich): tune. -const partiallyOrderedDistinctNumHashBuckets = 1024 - // newPartiallyOrderedDistinct creates a distinct operator on the given // distinct columns when we have partial ordering on some of the distinct // columns. @@ -63,10 +60,7 @@ func newPartiallyOrderedDistinct( distinctUnorderedCols = append(distinctUnorderedCols, distinctCol) } } - distinct := NewUnorderedDistinct( - allocator, chunkerOperator, distinctUnorderedCols, typs, - partiallyOrderedDistinctNumHashBuckets, - ) + distinct := NewUnorderedDistinct(allocator, chunkerOperator, distinctUnorderedCols, typs) return &partiallyOrderedDistinct{ input: chunkerOperator, distinct: distinct.(ResettableOperator), diff --git a/pkg/sql/colexec/routers.go b/pkg/sql/colexec/routers.go index 83b6b8ba63cb..a6b46945a03c 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -505,8 +505,6 @@ const ( // returned by the constructor. type HashRouter struct { OneInputNode - // types are the input types. - types []*types.T // hashCols is a slice of indices of the columns used for hashing. hashCols []uint32 @@ -596,12 +594,11 @@ func NewHashRouter( outputs[i] = op outputsAsOps[i] = op } - return newHashRouterWithOutputs(input, types, hashCols, unblockEventsChan, outputs, toDrain, toClose), outputsAsOps + return newHashRouterWithOutputs(input, hashCols, unblockEventsChan, outputs, toDrain, toClose), outputsAsOps } func newHashRouterWithOutputs( input colexecbase.Operator, - types []*types.T, hashCols []uint32, unblockEventsChan <-chan struct{}, outputs []routerOutput, @@ -610,7 +607,6 @@ func newHashRouterWithOutputs( ) *HashRouter { r := &HashRouter{ OneInputNode: NewOneInputNode(input), - types: types, hashCols: hashCols, outputs: outputs, closers: toClose, @@ -739,7 +735,7 @@ func (r *HashRouter) processNextBatch(ctx context.Context) bool { return true } - selections := r.tupleDistributor.distribute(ctx, b, r.types, r.hashCols) + selections := r.tupleDistributor.distribute(ctx, b, r.hashCols) for i, o := range r.outputs { if o.addBatch(ctx, b, selections[i]) { // This batch blocked the output. diff --git a/pkg/sql/colexec/routers_test.go b/pkg/sql/colexec/routers_test.go index d55a85d69bb3..327f76e87357 100644 --- a/pkg/sql/colexec/routers_test.go +++ b/pkg/sql/colexec/routers_test.go @@ -639,7 +639,6 @@ func TestHashRouterComputesDestination(t *testing.T) { expectedNumVals = []int{273, 252, 287, 212} numOutputs = 4 valsPushed = make([]int, numOutputs) - typs = []*types.T{types.Int} ) outputs := make([]routerOutput, numOutputs) @@ -666,7 +665,7 @@ func TestHashRouterComputesDestination(t *testing.T) { } } - r := newHashRouterWithOutputs(in, typs, []uint32{0}, nil /* ch */, outputs, nil /* toDrain */, nil /* toClose */) + r := newHashRouterWithOutputs(in, []uint32{0}, nil /* ch */, outputs, nil /* toDrain */, nil /* toClose */) for r.processNextBatch(ctx) { } @@ -707,7 +706,7 @@ func TestHashRouterCancellation(t *testing.T) { in := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) unbufferedCh := make(chan struct{}) - r := newHashRouterWithOutputs(in, typs, []uint32{0}, unbufferedCh, routerOutputs, nil /* toDrain */, nil /* toClose */) + r := newHashRouterWithOutputs(in, []uint32{0}, unbufferedCh, routerOutputs, nil /* toDrain */, nil /* toClose */) t.Run("BeforeRun", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) @@ -994,7 +993,6 @@ func TestHashRouterRandom(t *testing.T) { const hashRouterMetadataMsg = "hash router test metadata" r := newHashRouterWithOutputs( inputs[0], - typs, hashCols, unblockEventsChan, outputs, diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index c91087e7b911..4588e75b8772 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -24,15 +24,13 @@ import ( // numHashBuckets determines the number of buckets that the hash table is // created with. func NewUnorderedDistinct( - allocator *colmem.Allocator, - input colexecbase.Operator, - distinctCols []uint32, - typs []*types.T, - numHashBuckets uint64, + allocator *colmem.Allocator, input colexecbase.Operator, distinctCols []uint32, typs []*types.T, ) colexecbase.Operator { + // This number was chosen after running the micro-benchmarks. + const hashTableLoadFactor = 2.0 ht := newHashTable( allocator, - numHashBuckets, + hashTableLoadFactor, typs, distinctCols, true, /* allowNullEquality */ diff --git a/pkg/sql/distsql/columnar_operators_test.go b/pkg/sql/distsql/columnar_operators_test.go index 607dca667d5e..ecd56c6b1de6 100644 --- a/pkg/sql/distsql/columnar_operators_test.go +++ b/pkg/sql/distsql/columnar_operators_test.go @@ -627,7 +627,7 @@ func TestHashJoinerAgainstProcessor(t *testing.T) { rInputTypes[iColIdx], rInputTypes[jColIdx] = rInputTypes[jColIdx], rInputTypes[iColIdx] rEqCols[i], rEqCols[j] = rEqCols[j], rEqCols[i] }) - rInputTypes = generateRandomComparableTypes(rng, rInputTypes) + rInputTypes = randomizeJoinRightTypes(rng, rInputTypes) lRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, lInputTypes) rRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, rInputTypes) usingRandomTypes = true @@ -826,7 +826,7 @@ func TestMergeJoinerAgainstProcessor(t *testing.T) { rInputTypes[iColIdx], rInputTypes[jColIdx] = rInputTypes[jColIdx], rInputTypes[iColIdx] rOrderingCols[i], rOrderingCols[j] = rOrderingCols[j], rOrderingCols[i] }) - rInputTypes = generateRandomComparableTypes(rng, rInputTypes) + rInputTypes = randomizeJoinRightTypes(rng, rInputTypes) lRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, lInputTypes) rRows = sqlbase.RandEncDatumRowsOfTypes(rng, nRows, rInputTypes) usingRandomTypes = true @@ -1116,38 +1116,28 @@ func generateRandomSupportedTypes(rng *rand.Rand, nCols int) []*types.T { return typs } -// generateRandomComparableTypes generates random types that are supported by -// the vectorized engine and are such that they are comparable to the -// corresponding types in inputTypes. -func generateRandomComparableTypes(rng *rand.Rand, inputTypes []*types.T) []*types.T { - typs := make([]*types.T, len(inputTypes)) - for i, inputType := range inputTypes { - for { - typ := sqlbase.RandType(rng) - if typeconv.TypeFamilyToCanonicalTypeFamily(typ.Family()) == typeconv.DatumVecCanonicalTypeFamily { - // At the moment, we disallow datum-backed types. - // TODO(yuzefovich): remove this. - continue - } - comparable := false - for _, cmpOverloads := range tree.CmpOps[tree.LT] { - o := cmpOverloads.(*tree.CmpOp) - if inputType.Equivalent(o.LeftType) && typ.Equivalent(o.RightType) { - if (typ.Family() == types.DateFamily && inputType.Family() != types.DateFamily) || - (typ.Family() != types.DateFamily && inputType.Family() == types.DateFamily) { - // We map Dates to int64 and don't have casts from int64 to - // timestamps (and there is a comparison between dates and - // timestamps). - continue - } - comparable = true - break - } - } - if comparable { - typs[i] = typ - break +// randomizeJoinRightTypes returns somewhat random types to be used for the +// right side of the join such that they would have produced equality +// conditions in the non-test environment (currently, due to #43060, we don't +// support joins of different types without pushing the mixed-type equality +// checks into the ON condition). +func randomizeJoinRightTypes(rng *rand.Rand, leftTypes []*types.T) []*types.T { + typs := make([]*types.T, len(leftTypes)) + for i, inputType := range leftTypes { + switch inputType.Family() { + case types.IntFamily: + // We want to randomize integer types because they have different + // physical representations. + switch rng.Intn(3) { + case 0: + typs[i] = types.Int2 + case 1: + typs[i] = types.Int4 + default: + typs[i] = types.Int } + default: + typs[i] = inputType } } return typs diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 18b4d37d98fb..9953d3d67264 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -268,12 +268,13 @@ func TestPlanningDuringSplitsAndMerges(t *testing.T) { func TestDistSQLReceiverUpdatesCaches(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() size := func() int64 { return 2 << 10 } st := cluster.MakeTestingClusterSettings() rangeCache := kvcoord.NewRangeDescriptorCache(st, nil /* db */, size, stop.NewStopper()) r := MakeDistSQLReceiver( - context.Background(), nil /* resultWriter */, tree.Rows, + ctx, nil /* resultWriter */, tree.Rows, rangeCache, nil /* txn */, nil /* updateClock */, &SessionTracing{}) replicas := []roachpb.ReplicaDescriptor{{ReplicaID: 1}, {ReplicaID: 2}, {ReplicaID: 3}} @@ -323,7 +324,7 @@ func TestDistSQLReceiverUpdatesCaches(t *testing.T) { } for i := range descs { - ri := rangeCache.GetCached(descs[i].StartKey, false /* inclusive */) + ri := rangeCache.GetCached(ctx, descs[i].StartKey, false /* inclusive */) require.NotNilf(t, ri, "failed to find range for key: %s", descs[i].StartKey) require.Equal(t, &descs[i], ri.Desc()) require.NotNil(t, ri.Lease()) diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial_bbox b/pkg/sql/logictest/testdata/logic_test/geospatial_bbox index e8ecae4c69ff..53b54716a0e4 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial_bbox +++ b/pkg/sql/logictest/testdata/logic_test/geospatial_bbox @@ -172,6 +172,11 @@ BOX(-1 -1,1 1) BOX(4 -5,4 -5) BOX(-1 -5,4 1) +query T +select st_combinebbox(st_expand(NULL::BOX2D, 0.7845514859561931:::FLOAT8::FLOAT8)::BOX2D::BOX2D, '010200000000000000':::GEOMETRY::GEOMETRY)::BOX2D; +---- +NULL + subtest st_expand query TT diff --git a/pkg/sql/logictest/testdata/logic_test/hash_join b/pkg/sql/logictest/testdata/logic_test/hash_join index 736ff50ee9d6..b35524ea479d 100644 --- a/pkg/sql/logictest/testdata/logic_test/hash_join +++ b/pkg/sql/logictest/testdata/logic_test/hash_join @@ -208,3 +208,11 @@ SELECT * FROM t44797_2 NATURAL JOIN t44797_3 1 1.0 2 2.0 2 1.0 + +# Check that non-inner join with mixed-type equality considers the "same" +# values of different types as equal. +query R rowsort +SELECT * FROM t44797_2 WHERE EXISTS (SELECT * FROM t44797_2 AS l, t44797_3 AS r WHERE l.a = r.b) +---- +1 +2 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 2c8023c965b6..2739c98cae9a 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1045,7 +1045,7 @@ oid typname typnamespace typowner typlen typbyval typtype 90001 _geometry 1307062959 NULL -1 false b 90002 geography 1307062959 NULL -1 false b 90003 _geography 1307062959 NULL -1 false b -90004 box2d 1307062959 NULL 8 true b +90004 box2d 1307062959 NULL 32 true b 90005 _box2d 1307062959 NULL -1 false b 100064 newtype1 2332901747 NULL -1 false e 100065 _newtype1 2332901747 NULL -1 false b diff --git a/pkg/sql/sem/builtins/aggregate_builtins.go b/pkg/sql/sem/builtins/aggregate_builtins.go index 9daef08cecba..da3b6ea204cc 100644 --- a/pkg/sql/sem/builtins/aggregate_builtins.go +++ b/pkg/sql/sem/builtins/aggregate_builtins.go @@ -672,7 +672,7 @@ func (agg *stExtentAgg) Result() (tree.Datum, error) { if agg.bbox == nil { return tree.DNull, nil } - return tree.NewDBox2D(agg.bbox), nil + return tree.NewDBox2D(*agg.bbox), nil } // Reset implements the AggregateFunc interface. diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 769d51cca774..73d0c727bfec 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -407,10 +407,11 @@ var geoBuiltins = map[string]builtinDefinition{ defProps(), geometryOverload1( func(_ *tree.EvalContext, g *tree.DGeometry) (tree.Datum, error) { - if g.Geometry.Empty() { + bbox := g.CartesianBoundingBox() + if bbox == nil { return tree.DNull, nil } - return tree.NewDBox2D(g.CartesianBoundingBox()), nil + return tree.NewDBox2D(*bbox), nil }, types.Box2D, infoBuilder{ @@ -3681,9 +3682,8 @@ Bottom Left.`, if aGeomT.Empty() || bGeomT.Empty() { return nil, errors.Newf("cannot use POINT EMPTY") } - return tree.NewDBox2D( - a.CartesianBoundingBox().Combine(b.CartesianBoundingBox()), - ), nil + bbox := a.CartesianBoundingBox().Combine(b.CartesianBoundingBox()) + return tree.NewDBox2D(*bbox), nil default: return nil, errors.Newf("second argument is not a POINT") } @@ -3704,18 +3704,23 @@ Bottom Left.`, Types: tree.ArgTypes{{"box2d", types.Box2D}, {"geometry", types.Geometry}}, ReturnType: tree.FixedReturnType(types.Box2D), Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { - if args[0] == tree.DNull && args[1] == tree.DNull { - return tree.DNull, nil - } - if args[0] == tree.DNull { - return tree.NewDBox2D(tree.MustBeDGeometry(args[1]).CartesianBoundingBox()), nil - } if args[1] == tree.DNull { return args[0], nil } - bbox := tree.MustBeDBox2D(args[0]) + if args[0] == tree.DNull { + bbox := tree.MustBeDGeometry(args[1]).CartesianBoundingBox() + if bbox == nil { + return tree.DNull, nil + } + return tree.NewDBox2D(*bbox), nil + } + bbox := &tree.MustBeDBox2D(args[0]).CartesianBoundingBox g := tree.MustBeDGeometry(args[1]) - return tree.NewDBox2D(bbox.Combine(g.CartesianBoundingBox())), nil + bbox = bbox.Combine(g.CartesianBoundingBox()) + if bbox == nil { + return tree.DNull, nil + } + return tree.NewDBox2D(*bbox), nil }, Info: infoBuilder{ info: "Combines the current bounding box with the bounding box of the Geometry.", @@ -3731,7 +3736,11 @@ Bottom Left.`, Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { bbox := tree.MustBeDBox2D(args[0]) delta := float64(tree.MustBeDFloat(args[1])) - return tree.NewDBox2D(bbox.Buffer(delta, delta)), nil + bboxBuffered := bbox.Buffer(delta, delta) + if bboxBuffered == nil { + return tree.DNull, nil + } + return tree.NewDBox2D(*bboxBuffered), nil }, Info: infoBuilder{ info: "Extends the box2d by delta units across all dimensions.", @@ -3749,7 +3758,11 @@ Bottom Left.`, bbox := tree.MustBeDBox2D(args[0]) deltaX := float64(tree.MustBeDFloat(args[1])) deltaY := float64(tree.MustBeDFloat(args[2])) - return tree.NewDBox2D(bbox.Buffer(deltaX, deltaY)), nil + bboxBuffered := bbox.Buffer(deltaX, deltaY) + if bboxBuffered == nil { + return tree.DNull, nil + } + return tree.NewDBox2D(*bboxBuffered), nil }, Info: infoBuilder{ info: "Extends the box2d by delta_x units in the x dimension and delta_y units in the y dimension.", diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go index eff815d46ef1..38de9403be26 100644 --- a/pkg/sql/sem/tree/casts.go +++ b/pkg/sql/sem/tree/casts.go @@ -715,10 +715,11 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { case *DBox2D: return d, nil case *DGeometry: - if d.Geometry.Empty() { + bbox := d.CartesianBoundingBox() + if bbox == nil { return DNull, nil } - return NewDBox2D(d.CartesianBoundingBox()), nil + return NewDBox2D(*bbox), nil } case types.GeographyFamily: diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index f18099ae5fe7..22bf7c553aaf 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -2937,11 +2937,11 @@ func (d *DGeometry) Size() uintptr { // DBox2D is the Datum representation of the Box2D type. type DBox2D struct { - *geo.CartesianBoundingBox + geo.CartesianBoundingBox } // NewDBox2D returns a new Box2D Datum. -func NewDBox2D(b *geo.CartesianBoundingBox) *DBox2D { +func NewDBox2D(b geo.CartesianBoundingBox) *DBox2D { return &DBox2D{CartesianBoundingBox: b} } @@ -2990,7 +2990,7 @@ func (d *DBox2D) Compare(ctx *EvalContext, other Datum) int { return 1 } o := other.(*DBox2D) - return d.CartesianBoundingBox.Compare(o.CartesianBoundingBox) + return d.CartesianBoundingBox.Compare(&o.CartesianBoundingBox) } // Prev implements the Datum interface. @@ -3041,7 +3041,7 @@ func (d *DBox2D) Format(ctx *FmtCtx) { // Size implements the Datum interface. func (d *DBox2D) Size() uintptr { - return unsafe.Sizeof(*d) + unsafe.Sizeof(*d.CartesianBoundingBox) + return unsafe.Sizeof(*d) + unsafe.Sizeof(d.CartesianBoundingBox) } // DJSON is the JSON Datum. @@ -4651,7 +4651,7 @@ var baseDatumTypeSizes = map[types.Family]struct { }{ types.UnknownFamily: {unsafe.Sizeof(dNull{}), fixedSize}, types.BoolFamily: {unsafe.Sizeof(DBool(false)), fixedSize}, - types.Box2DFamily: {unsafe.Sizeof(DBox2D{CartesianBoundingBox: &geo.CartesianBoundingBox{}}), fixedSize}, + types.Box2DFamily: {unsafe.Sizeof(DBox2D{CartesianBoundingBox: geo.CartesianBoundingBox{}}), fixedSize}, types.BitFamily: {unsafe.Sizeof(DBitArray{}), variableSize}, types.IntFamily: {unsafe.Sizeof(DInt(0)), fixedSize}, types.FloatFamily: {unsafe.Sizeof(DFloat(0.0)), fixedSize}, diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index ee98911f3ac6..c851d00363fe 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -2559,8 +2559,8 @@ func makeBox2DComparisonOperators(op func(lhs, rhs *geo.CartesianBoundingBox) bo return nil, err } ret := op( - MustBeDBox2D(left).CartesianBoundingBox, - MustBeDBox2D(right).CartesianBoundingBox, + &MustBeDBox2D(left).CartesianBoundingBox, + &MustBeDBox2D(right).CartesianBoundingBox, ) return MakeDBool(DBool(ret)), nil }, @@ -2574,7 +2574,7 @@ func makeBox2DComparisonOperators(op func(lhs, rhs *geo.CartesianBoundingBox) bo return nil, err } ret := op( - MustBeDBox2D(left).CartesianBoundingBox, + &MustBeDBox2D(left).CartesianBoundingBox, MustBeDGeometry(right).CartesianBoundingBox(), ) return MakeDBool(DBool(ret)), nil @@ -2590,7 +2590,7 @@ func makeBox2DComparisonOperators(op func(lhs, rhs *geo.CartesianBoundingBox) bo } ret := op( MustBeDGeometry(left).CartesianBoundingBox(), - MustBeDBox2D(right).CartesianBoundingBox, + &MustBeDBox2D(right).CartesianBoundingBox, ) return MakeDBool(DBool(ret)), nil }, diff --git a/pkg/sql/sem/tree/testutils.go b/pkg/sql/sem/tree/testutils.go index aa0d906026a3..d526d3cc625d 100644 --- a/pkg/sql/sem/tree/testutils.go +++ b/pkg/sql/sem/tree/testutils.go @@ -82,7 +82,7 @@ func SampleDatum(t *types.T) Datum { return NewDOid(DInt(1009)) case types.Box2DFamily: b := geo.NewCartesianBoundingBox().AddPoint(1, 2).AddPoint(3, 4) - return NewDBox2D(b) + return NewDBox2D(*b) case types.GeographyFamily: return NewDGeography(geo.MustParseGeographyFromEWKB([]byte("\x01\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f"))) case types.GeometryFamily: diff --git a/pkg/sql/sqlbase/column_type_encoding.go b/pkg/sql/sqlbase/column_type_encoding.go index 5b9b74ec7575..d163f3d3b53d 100644 --- a/pkg/sql/sqlbase/column_type_encoding.go +++ b/pkg/sql/sqlbase/column_type_encoding.go @@ -292,7 +292,7 @@ func DecodeTableKey( } return a.NewDBytes(tree.DBytes(r)), rkey, err case types.Box2DFamily: - var r *geo.CartesianBoundingBox + var r geo.CartesianBoundingBox if dir == encoding.Ascending { rkey, r, err = encoding.DecodeBox2DAscending(key) } else { diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go index 45ab275745c9..4fceaf8dfde8 100644 --- a/pkg/sql/sqlbase/testutils.go +++ b/pkg/sql/sqlbase/testutils.go @@ -108,7 +108,7 @@ func RandDatumWithNullChance(rng *rand.Rand, typ *types.T, nullChance int) tree. } case types.Box2DFamily: b := geo.NewCartesianBoundingBox().AddPoint(rng.NormFloat64(), rng.NormFloat64()).AddPoint(rng.NormFloat64(), rng.NormFloat64()) - return tree.NewDBox2D(b) + return tree.NewDBox2D(*b) case types.GeographyFamily: gm, err := typ.GeoMetadata() if err != nil { @@ -513,7 +513,7 @@ var ( &tree.DInterval{Duration: duration.MakeDuration(0, 0, 290*12)}, }, types.Box2DFamily: { - &tree.DBox2D{CartesianBoundingBox: &geo.CartesianBoundingBox{BoundingBox: geopb.BoundingBox{LoX: -10, HiX: 10, LoY: -10, HiY: 10}}}, + &tree.DBox2D{CartesianBoundingBox: geo.CartesianBoundingBox{BoundingBox: geopb.BoundingBox{LoX: -10, HiX: 10, LoY: -10, HiY: 10}}}, }, types.GeographyFamily: { // NOTE(otan): we cannot use WKT here because roachtests do not have geos uploaded. diff --git a/pkg/util/cache/cache.go b/pkg/util/cache/cache.go index 2d10a7d95766..e080ff75cce2 100644 --- a/pkg/util/cache/cache.go +++ b/pkg/util/cache/cache.go @@ -459,6 +459,17 @@ func (oc *OrderedCache) DoRangeEntry(f func(e *Entry) bool, from, to interface{} }, &Entry{Key: from}, &Entry{Key: to}) } +// DoRangeReverseEntry invokes f on all cache entries in the range (to, from]. from +// should be higher than to. +// f returns a boolean indicating the traversal is done. If f returns true, the +// DoRangeReverseEntry loop will exit; false, it will continue. +// DoRangeReverseEntry returns whether the iteration exited early. +func (oc *OrderedCache) DoRangeReverseEntry(f func(e *Entry) bool, from, to interface{}) bool { + return oc.llrb.DoRangeReverse(func(e llrb.Comparable) bool { + return f(e.(*Entry)) + }, &Entry{Key: from}, &Entry{Key: to}) +} + // DoRange invokes f on all key-value pairs in the range of from -> to. f // returns a boolean indicating the traversal is done. If f returns true, the // DoRange loop will exit; false, it will continue. DoRange returns whether the diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go index 0f03030fd1e4..91f052886070 100644 --- a/pkg/util/encoding/encoding.go +++ b/pkg/util/encoding/encoding.go @@ -981,7 +981,7 @@ func decodeTime(b []byte) (r []byte, sec int64, nsec int64, err error) { } // EncodeBox2DAscending encodes a bounding box in ascending order. -func EncodeBox2DAscending(b []byte, box *geo.CartesianBoundingBox) ([]byte, error) { +func EncodeBox2DAscending(b []byte, box geo.CartesianBoundingBox) ([]byte, error) { b = append(b, box2DMarker) b = EncodeFloatAscending(b, box.LoX) b = EncodeFloatAscending(b, box.HiX) @@ -991,7 +991,7 @@ func EncodeBox2DAscending(b []byte, box *geo.CartesianBoundingBox) ([]byte, erro } // EncodeBox2DDescending encodes a bounding box in descending order. -func EncodeBox2DDescending(b []byte, box *geo.CartesianBoundingBox) ([]byte, error) { +func EncodeBox2DDescending(b []byte, box geo.CartesianBoundingBox) ([]byte, error) { b = append(b, box2DMarker) b = EncodeFloatDescending(b, box.LoX) b = EncodeFloatDescending(b, box.HiX) @@ -1001,57 +1001,57 @@ func EncodeBox2DDescending(b []byte, box *geo.CartesianBoundingBox) ([]byte, err } // DecodeBox2DAscending decodes a box2D object in ascending order. -func DecodeBox2DAscending(b []byte) ([]byte, *geo.CartesianBoundingBox, error) { +func DecodeBox2DAscending(b []byte) ([]byte, geo.CartesianBoundingBox, error) { + box := geo.CartesianBoundingBox{} if PeekType(b) != Box2D { - return nil, nil, errors.Errorf("did not find Box2D marker") + return nil, box, errors.Errorf("did not find Box2D marker") } b = b[1:] - box := &geo.CartesianBoundingBox{} var err error b, box.LoX, err = DecodeFloatAscending(b) if err != nil { - return nil, nil, err + return nil, box, err } b, box.HiX, err = DecodeFloatAscending(b) if err != nil { - return nil, nil, err + return nil, box, err } b, box.LoY, err = DecodeFloatAscending(b) if err != nil { - return nil, nil, err + return nil, box, err } b, box.HiY, err = DecodeFloatAscending(b) if err != nil { - return nil, nil, err + return nil, box, err } return b, box, nil } // DecodeBox2DDescending decodes a box2D object in descending order. -func DecodeBox2DDescending(b []byte) ([]byte, *geo.CartesianBoundingBox, error) { +func DecodeBox2DDescending(b []byte) ([]byte, geo.CartesianBoundingBox, error) { + box := geo.CartesianBoundingBox{} if PeekType(b) != Box2D { - return nil, nil, errors.Errorf("did not find Box2D marker") + return nil, box, errors.Errorf("did not find Box2D marker") } b = b[1:] - box := &geo.CartesianBoundingBox{} var err error b, box.LoX, err = DecodeFloatDescending(b) if err != nil { - return nil, nil, err + return nil, box, err } b, box.HiX, err = DecodeFloatDescending(b) if err != nil { - return nil, nil, err + return nil, box, err } b, box.LoY, err = DecodeFloatDescending(b) if err != nil { - return nil, nil, err + return nil, box, err } b, box.HiY, err = DecodeFloatDescending(b) if err != nil { - return nil, nil, err + return nil, box, err } return b, box, nil } @@ -2252,14 +2252,14 @@ func EncodeUntaggedTimeTZValue(appendTo []byte, t timetz.TimeTZ) []byte { // EncodeBox2DValue encodes a geo.CartesianBoundingBox with its value tag, appends it to // the supplied buffer and returns the final buffer. -func EncodeBox2DValue(appendTo []byte, colID uint32, b *geo.CartesianBoundingBox) ([]byte, error) { +func EncodeBox2DValue(appendTo []byte, colID uint32, b geo.CartesianBoundingBox) ([]byte, error) { appendTo = EncodeValueTag(appendTo, colID, Box2D) return EncodeUntaggedBox2DValue(appendTo, b) } // EncodeUntaggedBox2DValue encodes a geo.CartesianBoundingBox value, appends it to the supplied buffer, // and returns the final buffer. -func EncodeUntaggedBox2DValue(appendTo []byte, b *geo.CartesianBoundingBox) ([]byte, error) { +func EncodeUntaggedBox2DValue(appendTo []byte, b geo.CartesianBoundingBox) ([]byte, error) { appendTo = EncodeFloatAscending(appendTo, b.LoX) appendTo = EncodeFloatAscending(appendTo, b.HiX) appendTo = EncodeFloatAscending(appendTo, b.LoY) @@ -2554,25 +2554,25 @@ func DecodeDecimalValue(b []byte) (remaining []byte, d apd.Decimal, err error) { // DecodeUntaggedBox2DValue decodes a value encoded by EncodeUntaggedBox2DValue. func DecodeUntaggedBox2DValue( b []byte, -) (remaining []byte, box *geo.CartesianBoundingBox, err error) { - box = &geo.CartesianBoundingBox{} +) (remaining []byte, box geo.CartesianBoundingBox, err error) { + box = geo.CartesianBoundingBox{} remaining = b remaining, box.LoX, err = DecodeFloatAscending(remaining) if err != nil { - return b, nil, err + return b, box, err } remaining, box.HiX, err = DecodeFloatAscending(remaining) if err != nil { - return b, nil, err + return b, box, err } remaining, box.LoY, err = DecodeFloatAscending(remaining) if err != nil { - return b, nil, err + return b, box, err } remaining, box.HiY, err = DecodeFloatAscending(remaining) if err != nil { - return b, nil, err + return b, box, err } return remaining, box, err } diff --git a/pkg/util/encoding/encoding_test.go b/pkg/util/encoding/encoding_test.go index 1f1c423d4a8d..cffb33dce8a3 100644 --- a/pkg/util/encoding/encoding_test.go +++ b/pkg/util/encoding/encoding_test.go @@ -1150,10 +1150,10 @@ func TestEncodeDecodeTimeTZ(t *testing.T) { func TestEncodeDecodeBox2D(t *testing.T) { testCases := []struct { - ordered []*geo.CartesianBoundingBox + ordered []geo.CartesianBoundingBox }{ { - ordered: []*geo.CartesianBoundingBox{ + ordered: []geo.CartesianBoundingBox{ {BoundingBox: geopb.BoundingBox{LoX: -100, HiX: 99, LoY: -100, HiY: 100}}, {BoundingBox: geopb.BoundingBox{LoX: -100, HiX: 100, LoY: -100, HiY: 100}}, {BoundingBox: geopb.BoundingBox{LoX: -50, HiX: 100, LoY: -100, HiY: 100}}, @@ -1173,7 +1173,7 @@ func TestEncodeDecodeBox2D(t *testing.T) { for j := range tc.ordered { var b []byte var err error - var decoded *geo.CartesianBoundingBox + var decoded geo.CartesianBoundingBox if dir == Ascending { b, err = EncodeBox2DAscending(b, tc.ordered[j])