Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
63238: roachtest: update libpq blocklist to ignore TestCopyInBinaryError r=rafiss a=RichardJCai

roachtest: update libpq blocklist to ignore TestCopyInBinaryError

TestCopyInBinary's behaviour was incorrect in the test since we were not receiving an expected error (`pq: only text format supported for COPY`). 
Furthermore the test would sporadically panic causing the following tests to fail.

Release note: None

Resolves #57855 

63244: logictest: compare floating point values approximately on s390x r=ajwerner a=jonathan-albrecht-ibm

### Overview
On s390x in the std math package and some c-deps, floating point calculations can produce results that differ from the values calculated on amd64. This patch adds a function to compare logictest floating point and decimal values within a small relative margin on s390x. The existing behavior on all other platforms remains the same.

On s390x, there are three main reasons that floating point calculations sometimes give different results:
* the go compiler generates the s390x "fused multiply and add" (FMA) instruction where possible,
* the go math package uses s390x optimized versions of some functions,
* some c libs eg. libgeos, libproj also have platform specific floating point calculation differences.

### Proposal
The motivation for this work is so that users building CRDB on s390x do not need to diagnose tests that fail because of platform dependent floating point differences.

This PR proposes one possible approach to dealing with platform dependent floating point differences. Since development, testing and CI are done on amd64 it keeps the current logic for determining float equality exactly the same. On s390x, it determines values of decimal and float column types (R and F) in query tests to be equal if they are within a tolerance. See the new pkg/testutils/floatcmp package for the implementation of the approximate equality logic and changes in logictest.go to see how it is applied to only s390x.

There are probably other approaches I haven't thought of that would also work. I'd like to use this proposal to start a conversation on how all tests in CRDB that currently fail due to expected floating point differences could eventually be made to pass.

Of course platforms other than s390x may also have differences but I haven't looked at any other platforms. The changes should be easily extendable to other platforms if needed.

### Future Work
The changes in this PR allow the following tests to pass on s390x:
* TestLogic/fakedist-disk/builtin_function/extra_float_digits_3
*  TestLogic/fakedist-metadata/builtin_function/extra_float_digits_3
*  TestLogic/fakedist-vec-off/builtin_function/extra_float_digits_3
*  TestLogic/fakedist/builtin_function/extra_float_digits_3
*  TestLogic/local-spec-planning/builtin_function/extra_float_digits_3
*  TestLogic/local-vec-off/builtin_function/extra_float_digits_3
*  TestLogic/local/builtin_function/extra_float_digits_3

There are about 70 more tests that currently fail due to platform floating point differences on s390x, many are tests of geospatial functions. Assuming we can come up with a good approach, I'd like to continue working on fixes to be submitted in future PRs.

Release note: None

63802: colbuilder: optimize IS DISTINCT FROM NULL when null is casted r=yuzefovich a=yuzefovich

We have an optimized operator for `Is{Not}DistinctFrom` operation which
we can plan currently only if the right side is a constant NULL. In some
cases the optimizer might create a cast expression on the right in order
to propagate the type of the null, and previously we would fallback to
the default comparison operator in such scenario. This is suboptimal,
and this commit fixes the issue by special casing the scenario of
casting NULL to some type.

Fixes: #63792.

Release note: None

63903: sql: mark planNodeToRowSource as streaming intelligently r=yuzefovich a=yuzefovich

Previously, out of abundance of caution (and some laziness) we marked
all `planNodeToRowSource` processors as of "streaming" nature. This
marker influences whether we wrap it with a streaming or buffering
columnarizer into the vectorized flow. However, doing so is unnecessary
in most cases and kills some of the benefits of the vectorized model.
The only special planNode is `hookFnNode` which must be streaming, all
others are safe to have buffering around them. This commit implements
that idea. This required adding another method to `Processor` interface.

Release note: None

Co-authored-by: richardjcai <[email protected]>
Co-authored-by: Jonathan Albrecht <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
4 people committed Apr 20, 2021
5 parents b3b8fe0 + bb525e5 + 688583e + c441942 + 7b67857 commit d5672dd
Show file tree
Hide file tree
Showing 20 changed files with 458 additions and 131 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,7 @@ ALL_TESTS = [
"//pkg/storage/enginepb:enginepb_test",
"//pkg/storage/metamorphic:metamorphic_test",
"//pkg/storage:storage_test",
"//pkg/testutils/floatcmp:floatcmp_test",
"//pkg/testutils/keysutils:keysutils_test",
"//pkg/testutils/lint/passes/fmtsafe:fmtsafe_test",
"//pkg/testutils/lint/passes/forbiddenmethod:forbiddenmethod_test",
Expand Down
5 changes: 0 additions & 5 deletions pkg/ccl/backupccl/split_and_scatter_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,11 +158,6 @@ type splitAndScatterProcessor struct {

var _ execinfra.Processor = &splitAndScatterProcessor{}

// OutputTypes implements the execinfra.Processor interface.
func (ssp *splitAndScatterProcessor) OutputTypes() []*types.T {
return splitAndScatterOutputTypes
}

func newSplitAndScatterProcessor(
flowCtx *execinfra.FlowCtx,
processorID int32,
Expand Down
12 changes: 10 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_processors.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ import (

type changeAggregator struct {
execinfra.ProcessorBase
execinfra.StreamingProcessor

flowCtx *execinfra.FlowCtx
spec execinfrapb.ChangeAggregatorSpec
Expand Down Expand Up @@ -190,6 +189,11 @@ func newChangeAggregatorProcessor(
return ca, nil
}

// MustBeStreaming implements the execinfra.Processor interface.
func (ca *changeAggregator) MustBeStreaming() bool {
return true
}

// Start is part of the RowSource interface.
func (ca *changeAggregator) Start(ctx context.Context) {
ctx = ca.StartInternal(ctx, changeAggregatorProcName)
Expand Down Expand Up @@ -834,7 +838,6 @@ const (

type changeFrontier struct {
execinfra.ProcessorBase
execinfra.StreamingProcessor

flowCtx *execinfra.FlowCtx
spec execinfrapb.ChangeFrontierSpec
Expand Down Expand Up @@ -967,6 +970,11 @@ func newChangeFrontierProcessor(
return cf, nil
}

// MustBeStreaming implements the execinfra.Processor interface.
func (cf *changeFrontier) MustBeStreaming() bool {
return true
}

// Start is part of the RowSource interface.
func (cf *changeFrontier) Start(ctx context.Context) {
// StartInternal called at the beginning of the function because there are
Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/importccl/exportcsv.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,10 @@ func (sp *csvWriter) OutputTypes() []*types.T {
return res
}

func (sp *csvWriter) MustBeStreaming() bool {
return false
}

func (sp *csvWriter) Run(ctx context.Context) {
ctx, span := tracing.ChildSpan(ctx, "csvWriter")
defer span.Finish()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ const streamIngestionFrontierProcName = `ingestfntr`

type streamIngestionFrontier struct {
execinfra.ProcessorBase
execinfra.StreamingProcessor

flowCtx *execinfra.FlowCtx
spec execinfrapb.StreamIngestionFrontierSpec
Expand Down Expand Up @@ -86,6 +85,11 @@ func newStreamIngestionFrontierProcessor(
return sf, nil
}

// MustBeStreaming implements the execinfra.Processor interface.
func (sf *streamIngestionFrontier) MustBeStreaming() bool {
return true
}

// Start is part of the RowSource interface.
func (sf *streamIngestionFrontier) Start(ctx context.Context) {
ctx = sf.StartInternal(ctx, streamIngestionFrontierProcName)
Expand Down
78 changes: 51 additions & 27 deletions pkg/cmd/roachtest/libpq.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@ import (
"context"
"fmt"
"regexp"
"strings"

"github.com/stretchr/testify/require"
)

var libPQReleaseTagRegex = regexp.MustCompile(`^v(?P<major>\d+)\.(?P<minor>\d+)\.(?P<point>\d+)$`)
Expand All @@ -29,19 +32,14 @@ func registerLibPQ(r *testRegistry) {
c.Put(ctx, cockroach, "./cockroach", c.All())
c.Start(ctx, t, c.All())
version, err := fetchCockroachVersion(ctx, c, node[0])
if err != nil {
t.Fatal(err)
}
if err := alterZoneConfigAndClusterSettings(ctx, version, c, node[0]); err != nil {
t.Fatal(err)
}
require.NoError(t, err)
err = alterZoneConfigAndClusterSettings(ctx, version, c, node[0])
require.NoError(t, err)

t.Status("cloning lib/pq and installing prerequisites")
latestTag, err := repeatGetLatestTag(
ctx, c, "lib", "pq", libPQReleaseTagRegex)
if err != nil {
t.Fatal(err)
}
require.NoError(t, err)
c.l.Printf("Latest lib/pq release is %s.", latestTag)
c.l.Printf("Supported lib/pq release is %s.", libPQSupportedTag)

Expand All @@ -55,59 +53,85 @@ func registerLibPQ(r *testRegistry) {
)

// Remove any old lib/pq installations
if err := repeatRunE(
err = repeatRunE(
ctx, c, node, "remove old lib/pq", fmt.Sprintf("rm -rf %s", libPQPath),
); err != nil {
t.Fatal(err)
}
)
require.NoError(t, err)

// Install go-junit-report to convert test results to .xml format we know
// how to work with.
if err := repeatRunE(
ctx, c, node, "install go-junit-report", fmt.Sprintf("GOPATH=%s go get -u github.com/jstemmer/go-junit-report", goPath),
); err != nil {
t.Fatal(err)
}
err = repeatRunE(ctx, c, node, "install go-junit-report",
fmt.Sprintf("GOPATH=%s go get -u github.com/jstemmer/go-junit-report", goPath),
)
require.NoError(t, err)

if err := repeatGitCloneE(
err = repeatGitCloneE(
ctx,
t.l,
c,
fmt.Sprintf("https://%s.git", libPQRepo),
libPQPath,
libPQSupportedTag,
node,
); err != nil {
t.Fatal(err)
}

)
require.NoError(t, err)
_ = c.RunE(ctx, node, fmt.Sprintf("mkdir -p %s", resultsDir))

blocklistName, expectedFailures, ignorelistName, ignoredFailures := libPQBlocklists.getLists(version)
blocklistName, expectedFailures, ignorelistName, ignoreList := libPQBlocklists.getLists(version)
if expectedFailures == nil {
t.Fatalf("No lib/pq blocklist defined for cockroach version %s", version)
}
c.l.Printf("Running cockroach version %s, using blocklist %s, using ignorelist %s", version, blocklistName, ignorelistName)

t.Status("running lib/pq test suite and collecting results")

// List all the tests that start with Test or Example.
testListRegex := "^(Test|Example)"
buf, err := c.RunWithBuffer(
ctx,
t.l,
node,
fmt.Sprintf(`cd %s && PGPORT=26257 PGUSER=root PGSSLMODE=disable PGDATABASE=postgres go test -list "%s"`, libPQPath, testListRegex),
)
require.NoError(t, err)

// Convert the output of go test -list into an list.
tests := strings.Fields(string(buf))
var allowedTests []string

for _, testName := range tests {
// Ignore tests that do not match the test regex pattern.
matched, err := regexp.MatchString(testListRegex, testName)
require.NoError(t, err)
if !matched {
continue
}
// If the test is part of ignoreList, do not run the test.
if _, ok := ignoreList[testName]; !ok {
allowedTests = append(allowedTests, testName)
}
}

allowedTestsRegExp := fmt.Sprintf(`"^(%s)$"`, strings.Join(allowedTests, "|"))

// Ignore the error as there will be failing tests.
_ = c.RunE(
ctx,
node,
fmt.Sprintf("cd %s && PGPORT=26257 PGUSER=root PGSSLMODE=disable PGDATABASE=postgres go test -v 2>&1 | %s/bin/go-junit-report > %s", libPQPath, goPath, resultsPath),
fmt.Sprintf("cd %s && PGPORT=26257 PGUSER=root PGSSLMODE=disable PGDATABASE=postgres go test -run %s -v 2>&1 | %s/bin/go-junit-report > %s",
libPQPath, allowedTestsRegExp, goPath, resultsPath),
)

parseAndSummarizeJavaORMTestsResults(
ctx, t, c, node, "lib/pq" /* ormName */, []byte(resultsPath),
blocklistName, expectedFailures, ignoredFailures, version, latestTag,
blocklistName, expectedFailures, ignoreList, version, latestTag,
)
}

r.Add(testSpec{
Name: "lib/pq",
Owner: OwnerSQLExperience,
MinVersion: "v19.2.0",
MinVersion: "v20.1.0",
Cluster: makeClusterSpec(1),
Tags: []string{`default`, `driver`},
Run: runLibPQ,
Expand Down
83 changes: 10 additions & 73 deletions pkg/cmd/roachtest/libpq_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
package main

var libPQBlocklists = blocklistsForVersion{
{"v19.2", "libPQBlocklist19_2", libPQBlocklist19_2, "libPQIgnorelist19_2", libPQIgnorelist19_2},
{"v20.1", "libPQBlocklist20_1", libPQBlocklist20_1, "libPQIgnorelist20_1", libPQIgnorelist20_1},
{"v20.2", "libPQBlocklist20_2", libPQBlocklist20_2, "libPQIgnorelist20_2", libPQIgnorelist20_2},
{"v21.1", "libPQBlocklist21_1", libPQBlocklist21_1, "libPQIgnorelist21_1", libPQIgnorelist21_1},
Expand All @@ -33,7 +32,6 @@ var libPQBlocklist20_2 = blocklist{
"pq.TestContextCancelExec": "41335",
"pq.TestContextCancelQuery": "41335",
"pq.TestCopyFromError": "5807",
"pq.TestCopyInBinaryError": "5807",
"pq.TestCopyInRaiseStmtTrigger": "5807",
"pq.TestCopyInTypes": "5807",
"pq.TestCopyRespLoopConnectionError": "5807",
Expand Down Expand Up @@ -71,7 +69,6 @@ var libPQBlocklist20_1 = blocklist{
"pq.TestContextCancelExec": "41335",
"pq.TestContextCancelQuery": "41335",
"pq.TestCopyFromError": "5807",
"pq.TestCopyInBinaryError": "5807",
"pq.TestCopyInRaiseStmtTrigger": "5807",
"pq.TestCopyInTypes": "5807",
"pq.TestCopyRespLoopConnectionError": "5807",
Expand Down Expand Up @@ -107,81 +104,21 @@ var libPQBlocklist20_1 = blocklist{
"pq.TestTimeWithoutTimezone/24:00_=>_0000-01-02T00:00:00Z": "44548",
}

var libPQBlocklist19_2 = blocklist{
"pq.ExampleConnectorWithNoticeHandler": "unknown",
"pq.TestBinaryByteSliceToInt": "41547",
"pq.TestBinaryByteSlicetoUUID": "41547",
"pq.TestBindError": "5807",
"pq.TestByteaOutputFormats": "26947",
"pq.TestCommit": "5807",
"pq.TestConnListen": "41522",
"pq.TestConnUnlisten": "41522",
"pq.TestConnUnlistenAll": "41522",
"pq.TestConnectorWithNoticeHandler_Simple": "unknown",
"pq.TestConnectorWithNotificationHandler_Simple": "unknown",
"pq.TestContextCancelBegin": "41335",
"pq.TestContextCancelExec": "41335",
"pq.TestContextCancelQuery": "41335",
"pq.TestCopyFromError": "5807",
"pq.TestCopyInBinaryError": "5807",
"pq.TestCopyInMultipleValues": "5807",
"pq.TestCopyInRaiseStmtTrigger": "5807",
"pq.TestCopyInStmtAffectedRows": "5807",
"pq.TestCopyInTypes": "5807",
"pq.TestCopyInWrongType": "5807",
"pq.TestCopyRespLoopConnectionError": "5807",
"pq.TestEncodeAndParseTs": "41563",
"pq.TestErrorDuringStartup": "41551",
"pq.TestErrorOnExec": "5807",
"pq.TestErrorOnQuery": "5807",
"pq.TestErrorOnQueryRowSimpleQuery": "5807",
"pq.TestExec": "5807",
"pq.TestInfinityTimestamp": "41564",
"pq.TestIssue186": "41558",
"pq.TestIssue196": "41689",
"pq.TestIssue282": "12137",
"pq.TestIssue494": "5807",
"pq.TestListenerFailedQuery": "41522",
"pq.TestListenerListen": "41522",
"pq.TestListenerReconnect": "41522",
"pq.TestListenerUnlisten": "41522",
"pq.TestListenerUnlistenAll": "41522",
"pq.TestNotifyExtra": "41522",
"pq.TestPing": "35897",
"pq.TestQueryRowBugWorkaround": "5807",
"pq.TestReconnect": "35897",
"pq.TestReturning": "5807",
"pq.TestRowsColumnTypes": "41688",
"pq.TestRowsResultTag": "5807",
"pq.TestRuntimeParameters": "12137",
"pq.TestStringWithNul": "26366",
"pq.TestTimeWithTimezone": "44548",
"pq.TestTimeWithTimezone/11:59:59+00:00_=>_0000-01-01T11:59:59Z": "44548",
"pq.TestTimeWithTimezone/11:59:59+04:00_=>_0000-01-01T11:59:59+04:00": "44548",
"pq.TestTimeWithTimezone/24:00+00_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithTimezone/24:00-04:00_=>_0000-01-02T00:00:00-04:00": "44548",
"pq.TestTimeWithTimezone/24:00:00+00_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithTimezone/24:00:00.0+00_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithTimezone/24:00:00.000000+00_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithTimezone/24:00Z_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithoutTimezone": "44548",
"pq.TestTimeWithoutTimezone/24:00:00.000000_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithoutTimezone/24:00:00.0_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithoutTimezone/24:00:00_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimeWithoutTimezone/24:00_=>_0000-01-02T00:00:00Z": "44548",
"pq.TestTimestampWithTimeZone": "41565",
}

var libPQIgnorelist21_1 = libPQIgnorelist20_2

var libPQIgnorelist20_2 = libPQIgnorelist20_1

var libPQIgnorelist20_1 = libPQIgnorelist19_2

var libPQIgnorelist19_2 = blocklist{
// The test names here do not include "pq." since `go test -list` returns
// the test name without "pq.". We use the name returned from `go test -list`
// to ignore the test.
var libPQIgnorelist20_1 = blocklist{
// TestFormatTsBacked fails due to not returning an error for accepting a
// timestamp format that postgres does not.
"pq.TestFormatTsBackend": "41690",
"TestFormatTsBackend": "41690",
// TestTxOptions fails because it attempts to change isolation levels.
"pq.TestTxOptions": "41690",
"TestTxOptions": "41690",
// TestCopyInBinaryError is expected to error with:
// pq: only text format supported for COPY, however no error is returned
// for CRDB.
"TestCopyInBinaryError": "63235",
}
21 changes: 20 additions & 1 deletion pkg/sql/colexec/colbuilder/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,8 +118,12 @@ func wrapRowSources(
return nil, releasables, err
}

proc, isProcessor := toWrap.(execinfra.Processor)
if !isProcessor {
return nil, nil, errors.AssertionFailedf("unexpectedly %T is not an execinfra.Processor", toWrap)
}
var c *colexec.Columnarizer
if _, mustBeStreaming := toWrap.(execinfra.StreamingProcessor); mustBeStreaming {
if proc.MustBeStreaming() {
c, err = colexec.NewStreamingColumnarizer(
ctx, colmem.NewAllocator(ctx, args.StreamingMemAccount, factory), flowCtx, args.Spec.ProcessorID, toWrap,
)
Expand Down Expand Up @@ -2161,6 +2165,21 @@ func planProjectionExpr(
}
right = tupleDatum
}
// We have a special case behavior for Is{Not}DistinctFrom before
// checking whether the right expression is constant below in order to
// extract NULL from the cast expression.
//
// Normally, the optimizer folds all constants; however, for nulls it
// creates a cast expression from tree.DNull to the desired type in
// order to propagate the type of the null. We need to extract the
// constant NULL so that the optimized operator was planned below.
if projOp == tree.IsDistinctFrom || projOp == tree.IsNotDistinctFrom {
if cast, ok := right.(*tree.CastExpr); ok {
if cast.Expr == tree.DNull {
right = tree.DNull
}
}
}
if rConstArg, rConst := right.(tree.Datum); rConst {
// Case 2: The right is constant.
// The projection result will be outputted to a new column which is
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/execinfra/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ type RowSource interface {
// RowSourcedProcessor is the union of RowSource and Processor.
type RowSourcedProcessor interface {
RowSource
Run(context.Context)
Processor
}

// Run reads records from the source and outputs them to the receiver, properly
Expand Down
Loading

0 comments on commit d5672dd

Please sign in to comment.