Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
106552: tests: support float approximation in roachtest query comparison utils r=rharding6373 a=rharding6373

tests, logictest, floatcmp: refactor comparison test util functions
    
This commit moves some float comparison test util functions from
logictest into the floatcmp package. It also moves a query result
comparison function from the tlp file to query_comparison_util in the
tests package.
    
This commit also marks roachtests as testonly targets.
    
Epic: none
    
Release note: None


tests: support float approximation in roachtest query comparison utils
    
Before this change unoptimized query oracle tests would compare results
using simple string comparison. However, due to floating point precision
limitations, it's possible for results with floating point to diverge
during the course of normal computation. This results in test failures
that are difficult to reproduce or determine whether they are expected
behavior.
    
This change utilizes existing floating point comparison functions used
by logic tests to match float values only to a specific precision. Like
the logic tests, we also have special handling for floats and decimals
under the s390x architecture (see #63244). In order to avoid costly
comparisons, we only check floating point precision if the naiive string
comparison approach fails and there are float or decimal types in the
result.
    
Epic: None
Fixes: #95665
    
Release note: None

106607: pkg/util/log: introduce `fluent.sink.conn.errors` metric to log package r=knz a=abarganier

Logging is a critical subsystem within CRDB, but as things
are today, we have very little observability into logging
itself. For starters, we have no metrics in the logging
package at all!

This makes it difficult to observe things within the log
package. For example, if a fluent-server log sink fails
to connect to FluentBit, how can we tell? We get some STDOUT
message, but that's about it.

It's time to get some metrics into the log package.

Doing so is a bit of a tricky dance, because pretty much every
package in CRDB imports the logging package, meaning you
almost always get a circular dependency when trying to make
use of any library within pkg/util/log. Therefore, this PR 
injects metrics functionality into the logging package.
It does so via a new interface called `LogMetrics` with
functions that enable its users to modify metrics.

The implementation of the interface can live elsewhere,
such as the metrics package itself, whe circular dependencies
aren't such a pain. We can then inject the implementation
into the log package.

With all that plumbing done, the PR also introduces a new metric,
`fluent.sink.conn.errors` representing fluentbit connection errors 
whenever a fluent-server log sink fails to establish a connection.

We can see the metric represented below in a test, where no
fluent-server was running for a moment, before starting it.
Note that the connection errors level off once the server was 
started:
<img width="1018" alt="Screenshot 2023-07-11 at 1 32 57 PM" src="https://github.com/cockroachdb/cockroach/assets/8194877/ccacf84e-e585-4a76-98af-ed145629f9ef">

Release note (ops change): This patch introduces the counter
metric `fluent.sink.conn.errors` to the CockroachDB tsdb,
which is exposed to `/_status/vars` clients as
`fluent_sink_conn_errors`. The metric is incremented whenever
a `fluent-server` log sink fails to establish a connection to
the log sink pointed to by the `address` for the sink in the
provided log config.

Epic: CC-9681

Addresses: #102753

107453: Revert "github-pull-request-make: temporary workaround" r=postamar a=postamar

This reverts commit 2bd61c0.

Relates to #106920.

Co-authored-by: rharding6373 <[email protected]>
Co-authored-by: Alex Barganier <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
  • Loading branch information
4 people committed Jul 24, 2023
4 parents 771d3f7 + c9999ae + abd9c99 + 13f1ab0 commit 06fb4c1
Show file tree
Hide file tree
Showing 27 changed files with 745 additions and 169 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -655,6 +655,7 @@ ALL_TESTS = [
"//pkg/util/log/eventpb:eventpb_test",
"//pkg/util/log/logconfig:logconfig_test",
"//pkg/util/log/logcrash:logcrash_test",
"//pkg/util/log/logmetrics:logmetrics_test",
"//pkg/util/log/testshout:testshout_test",
"//pkg/util/log:log_test",
"//pkg/util/metric/aggmetric:aggmetric_test",
Expand Down Expand Up @@ -2320,6 +2321,8 @@ GO_TARGETS = [
"//pkg/util/log/logcrash:logcrash",
"//pkg/util/log/logcrash:logcrash_test",
"//pkg/util/log/logflags:logflags",
"//pkg/util/log/logmetrics:logmetrics",
"//pkg/util/log/logmetrics:logmetrics_test",
"//pkg/util/log/logpb:logpb",
"//pkg/util/log/logtestutils:logtestutils",
"//pkg/util/log/logutil:logutil",
Expand Down
7 changes: 0 additions & 7 deletions pkg/cmd/github-pull-request-make/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,13 +209,6 @@ func main() {

if len(pkgs) > 0 {
for name, pkg := range pkgs {
// TODO(postamar): remove this temporary workaround
// This hack was added to get #106743 over the finish line.
if strings.HasPrefix(name, "pkg/ccl/schemachangerccl") ||
strings.HasPrefix(name, "pkg/sql/schemachanger") {
continue
}

// 20 minutes total seems OK, but at least 2 minutes per test.
// This should be reduced. See #46941.
duration := (20 * time.Minute) / time.Duration(len(pkgs))
Expand Down
3 changes: 3 additions & 0 deletions pkg/cmd/roachtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test")

go_library(
name = "roachtest_lib",
testonly = 1,
srcs = [
"cluster.go",
"github.go",
Expand Down Expand Up @@ -62,13 +63,15 @@ go_library(

go_binary(
name = "roachtest",
testonly = 1,
embed = [":roachtest_lib"],
visibility = ["//visibility:public"],
)

go_test(
name = "roachtest_test",
size = "small",
testonly = 1,
srcs = [
"cluster_test.go",
"github_test.go",
Expand Down
4 changes: 4 additions & 0 deletions pkg/cmd/roachtest/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "tests",
testonly = 1,
srcs = [
"acceptance.go",
"activerecord.go",
Expand Down Expand Up @@ -224,6 +225,7 @@ go_library(
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/floatcmp",
"//pkg/testutils/jobutils",
"//pkg/testutils/release",
"//pkg/testutils/sqlutils",
Expand Down Expand Up @@ -286,6 +288,7 @@ go_test(
srcs = [
"blocklist_test.go",
"drt_test.go",
"query_comparison_util_test.go",
"tpcc_test.go",
"util_load_group_test.go",
":mocks_drt", # keep
Expand All @@ -298,6 +301,7 @@ go_test(
"//pkg/roachprod/logger",
"//pkg/roachprod/prometheus",
"//pkg/testutils/skip",
"//pkg/util/leaktest",
"//pkg/util/version",
"@com_github_golang_mock//gomock",
"@com_github_google_go_github//github",
Expand Down
6 changes: 5 additions & 1 deletion pkg/cmd/roachtest/tests/costfuzz.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,11 @@ func runCostFuzzQuery(qgen queryGenerator, rnd *rand.Rand, h queryComparisonHelp
return nil
}

if diff := unsortedMatricesDiff(controlRows, perturbRows); diff != "" {
diff, err := unsortedMatricesDiffWithFloatComp(controlRows, perturbRows, h.colTypes)
if err != nil {
return err
}
if diff != "" {
// We have a mismatch in the perturbed vs control query outputs.
h.logStatements()
h.logVerboseOutput()
Expand Down
117 changes: 117 additions & 0 deletions pkg/cmd/roachtest/tests/query_comparison_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ import (
"math/rand"
"os"
"path/filepath"
"runtime"
"sort"
"strings"
"time"

Expand All @@ -28,10 +30,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
"github.com/cockroachdb/cockroach/pkg/internal/workloadreplay"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/testutils/floatcmp"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/google/go-cmp/cmp"
)

const (
Expand Down Expand Up @@ -424,6 +428,7 @@ type queryComparisonHelper struct {

statements []string
statementsAndExplains []sqlAndOutput
colTypes []string
}

// runQuery runs the given query and returns the output. If the stmt doesn't
Expand All @@ -448,6 +453,14 @@ func (h *queryComparisonHelper) runQuery(stmt string) ([][]string, error) {
return nil, err
}
defer rows.Close()
cts, err := rows.ColumnTypes()
if err != nil {
return nil, err
}
h.colTypes = make([]string, len(cts))
for i, ct := range cts {
h.colTypes[i] = ct.DatabaseTypeName()
}
return sqlutils.RowsToStrMatrix(rows)
}

Expand Down Expand Up @@ -504,3 +517,107 @@ func (h *queryComparisonHelper) logVerboseOutput() {
func (h *queryComparisonHelper) makeError(err error, msg string) error {
return errors.Wrapf(err, "%s. %d statements run", msg, h.stmtNo)
}

func joinAndSortRows(rowMatrix1, rowMatrix2 [][]string, sep string) (rows1, rows2 []string) {
for _, row := range rowMatrix1 {
rows1 = append(rows1, strings.Join(row[:], sep))
}
for _, row := range rowMatrix2 {
rows2 = append(rows2, strings.Join(row[:], sep))
}
sort.Strings(rows1)
sort.Strings(rows2)
return rows1, rows2
}

// unsortedMatricesDiffWithFloatComp sorts and compares the rows in rowMatrix1
// to rowMatrix2 and outputs a diff or message related to the comparison. If a
// string comparison of the rows fails, and they contain floats or decimals, it
// performs an approximate comparison of the values.
func unsortedMatricesDiffWithFloatComp(
rowMatrix1, rowMatrix2 [][]string, colTypes []string,
) (string, error) {
rows1, rows2 := joinAndSortRows(rowMatrix1, rowMatrix2, ",")
result := cmp.Diff(rows1, rows2)
if result == "" {
return result, nil
}
if len(rows1) != len(rows2) || len(colTypes) != len(rowMatrix1[0]) || len(colTypes) != len(rowMatrix2[0]) {
return result, nil
}
var needApproxMatch bool
for i := range colTypes {
// On s390x, check that values for both float and decimal coltypes are
// approximately equal to take into account platform differences in floating
// point calculations. On other architectures, check float values only.
if (runtime.GOARCH == "s390x" && colTypes[i] == "DECIMAL") ||
colTypes[i] == "FLOAT4" || colTypes[i] == "FLOAT8" {
needApproxMatch = true
break
}
}
if !needApproxMatch {
return result, nil
}
// Use an unlikely string as a separator so that we can make a comparison
// using sorted rows. We don't use the rows sorted above because splitting
// the rows could be ambiguous.
sep := ",unsortedMatricesDiffWithFloatComp separator,"
rows1, rows2 = joinAndSortRows(rowMatrix1, rowMatrix2, sep)
for i := range rows1 {
// Split the sorted rows.
row1 := strings.Split(rows1[i], sep)
row2 := strings.Split(rows2[i], sep)

for j := range row1 {
if runtime.GOARCH == "s390x" && colTypes[j] == "DECIMAL" {
// On s390x, check that values for both float and decimal coltypes are
// approximately equal to take into account platform differences in floating
// point calculations. On other architectures, check float values only.
match, err := floatcmp.FloatsMatchApprox(row1[j], row2[j])
if err != nil {
return "", err
}
if !match {
return result, nil
}
} else if colTypes[j] == "FLOAT4" || colTypes[j] == "FLOAT8" {
// Check that float values are approximately equal.
var err error
var match bool
if runtime.GOARCH == "s390x" {
match, err = floatcmp.FloatsMatchApprox(row1[j], row2[j])
} else {
match, err = floatcmp.FloatsMatch(row1[j], row2[j])
}
if err != nil {
return "", err
}
if !match {
return result, nil
}
} else {
// Check that other columns are equal with a string comparison.
if row1[j] != row2[j] {
return result, nil
}
}
}
}
return "", nil
}

// unsortedMatricesDiff sorts and compares rows of data.
func unsortedMatricesDiff(rowMatrix1, rowMatrix2 [][]string) string {
var rows1 []string
for _, row := range rowMatrix1 {
rows1 = append(rows1, strings.Join(row[:], ","))
}
var rows2 []string
for _, row := range rowMatrix2 {
rows2 = append(rows2, strings.Join(row[:], ","))
}
sort.Strings(rows1)
sort.Strings(rows2)
return cmp.Diff(rows1, rows2)
}
140 changes: 140 additions & 0 deletions pkg/cmd/roachtest/tests/query_comparison_util_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package tests

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

// TestUnsortedMatricesDiff is a unit test for the
// unsortedMatricesDiffWithFloatComp() and unsortedMatricesDiff() utility
// functions.
func TestUnsortedMatricesDiff(t *testing.T) {
defer leaktest.AfterTest(t)()
tcs := []struct {
name string
colTypes []string
t1, t2 [][]string
exactMatch bool
approxMatch bool
}{
{
name: "float exact match",
colTypes: []string{"FLOAT8"},
t1: [][]string{{"1.2345678901234567"}},
t2: [][]string{{"1.2345678901234567"}},
exactMatch: true,
},
{
name: "float approx match",
colTypes: []string{"FLOAT8"},
t1: [][]string{{"1.2345678901234563"}},
t2: [][]string{{"1.2345678901234564"}},
exactMatch: false,
approxMatch: true,
},
{
name: "float no match",
colTypes: []string{"FLOAT8"},
t1: [][]string{{"1.234567890123"}},
t2: [][]string{{"1.234567890124"}},
exactMatch: false,
approxMatch: false,
},
{
name: "multi float approx match",
colTypes: []string{"FLOAT8", "FLOAT8"},
t1: [][]string{{"1.2345678901234567", "1.2345678901234567"}},
t2: [][]string{{"1.2345678901234567", "1.2345678901234568"}},
exactMatch: false,
approxMatch: true,
},
{
name: "string no match",
colTypes: []string{"STRING"},
t1: [][]string{{"hello"}},
t2: [][]string{{"world"}},
exactMatch: false,
approxMatch: false,
},
{
name: "mixed types match",
colTypes: []string{"STRING", "FLOAT8"},
t1: [][]string{{"hello", "1.2345678901234567"}},
t2: [][]string{{"hello", "1.2345678901234567"}},
exactMatch: true,
},
{
name: "mixed types float approx match",
colTypes: []string{"STRING", "FLOAT8"},
t1: [][]string{{"hello", "1.23456789012345678"}},
t2: [][]string{{"hello", "1.23456789012345679"}},
exactMatch: false,
approxMatch: true,
},
{
name: "mixed types no match",
colTypes: []string{"STRING", "FLOAT8"},
t1: [][]string{{"hello", "1.2345678901234567"}},
t2: [][]string{{"world", "1.2345678901234567"}},
exactMatch: false,
approxMatch: false,
},
{
name: "different col count",
colTypes: []string{"STRING"},
t1: [][]string{{"hello", "1.2345678901234567"}},
t2: [][]string{{"world", "1.2345678901234567"}},
exactMatch: false,
approxMatch: false,
},
{
name: "different row count",
colTypes: []string{"STRING", "FLOAT8"},
t1: [][]string{{"hello", "1.2345678901234567"}, {"aloha", "2.345"}},
t2: [][]string{{"world", "1.2345678901234567"}},
exactMatch: false,
approxMatch: false,
},
{
name: "multi row unsorted",
colTypes: []string{"STRING", "FLOAT8"},
t1: [][]string{{"hello", "1.2345678901234567"}, {"world", "1.2345678901234560"}},
t2: [][]string{{"world", "1.2345678901234560"}, {"hello", "1.2345678901234567"}},
exactMatch: true,
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
match := unsortedMatricesDiff(tc.t1, tc.t2)
if tc.exactMatch && match != "" {
t.Fatalf("unsortedMatricesDiff: expected exact match, got diff: %s", match)
} else if !tc.exactMatch && match == "" {
t.Fatalf("unsortedMatricesDiff: expected no exact match, got no diff")
}

var err error
match, err = unsortedMatricesDiffWithFloatComp(tc.t1, tc.t2, tc.colTypes)
if err != nil {
t.Fatal(err)
}
if tc.exactMatch && match != "" {
t.Fatalf("unsortedMatricesDiffWithFloatComp: expected exact match, got diff: %s", match)
} else if !tc.exactMatch && tc.approxMatch && match != "" {
t.Fatalf("unsortedMatricesDiffWithFloatComp: expected approx match, got diff: %s", match)
} else if !tc.exactMatch && !tc.approxMatch && match == "" {
t.Fatalf("unsortedMatricesDiffWithFloatComp: expected no approx match, got no diff")
}
})
}
}
Loading

0 comments on commit 06fb4c1

Please sign in to comment.