Commit 77419519 authored by craig[bot]'s avatar craig[bot]

Merge #49851

49851: rowcontainer: fix hash row container for some types r=yuzefovich a=yuzefovich

The explanation is that `HashDiskRowContainer` is implemented using
`DiskRowContainer` with the equality columns (i.e. the columns to hash)
of the former being the ordering columns for the latter, and those
ordering columns are used to compute the keys of the rows (in
`encodeRow`) so that we could store the row in the sorted order. This
way we store the build (right) side of the join, but for the probe
(left) side we use `hashMemRowIterator` to compute the key of the
probing row. The key computation methods must be the same in both
places, otherwise, the results of the join can be incorrect. #45229
broke this synchronization by changing the key computation method in
`hashMemRowIterator.computeKey` to use `Fingerprint`. So we have to either
use `Fingerprint` in `encodeRow` or use `Encode` in `computeKey`. The first
choice doesn't seem to work because `Fingerprint` doesn't provide the
ordering we need in `DiskRowContainer`, so we need to use the second approach.

The ordering property is necessary because `DiskRowContainer` implements
"hash row container" by sorting all rows on the ordering (i.e. hash) columns
and using the ordering property to provide the "hashing" behavior (i.e. we
would seek to the first row that has the same hash columns and then iterate
from that row one row at a time forward until the hash columns remain the
same).  If we don't have the ordering property, then the necessary invariant
that all rows that hash to the same value are contiguous is not maintained.

Release note: None
Co-authored-by: default avatarYahor Yuzefovich <[email protected]>
parents 432a1c53 56244c4f
......@@ -121,7 +121,12 @@ func encodeColumnsOfRow(
if row[colIdx].IsNull() && !encodeNull {
return nil, true, nil
}
appendTo, err = row[colIdx].Fingerprint(colTypes[i], da, appendTo)
// Note: we cannot compare VALUE encodings because they contain column IDs
// which can vary.
// TODO(radu): we should figure out what encoding is readily available and
// use that (though it needs to be consistent across all rows). We could add
// functionality to compare VALUE encodings ignoring the column ID.
appendTo, err = row[colIdx].Encode(colTypes[i], da, sqlbase.DatumEncoding_ASCENDING_KEY, appendTo)
if err != nil {
return appendTo, false, err
}
......@@ -422,7 +427,7 @@ func (i *hashMemRowIterator) computeKey() error {
i.curKey = i.curKey[:0]
for _, col := range i.storedEqCols {
var err error
i.curKey, err = row[col].Fingerprint(i.types[col], &i.columnEncoder.datumAlloc, i.curKey)
i.curKey, err = row[col].Encode(i.types[col], &i.columnEncoder.datumAlloc, sqlbase.DatumEncoding_ASCENDING_KEY, i.curKey)
if err != nil {
return err
}
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment