From 7b20564d36d0cf4de2b81d197f3521e2363cf44a Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 29 Sep 2022 11:03:25 +0000 Subject: [PATCH 1/2] storage: avoid `EncodeMVCCValue` struct comparison MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In Go 1.19, struct comparisons saw a significant performance regression, apparently due to the use of `memeqbody`. This patch changes `EncodeMVCCValue` to use field comparisons instead of struct comparisons, which yields a significant performance gain. Unfortunately, this prevents mid-stack inlining. However, the struct comparison regression is significantly larger than the inlining gains. We should reconsider this once Go 1.20 lands, where the regression has been fixed. ``` name old time/op new time/op delta EncodeMVCCValue/header=empty/value=tombstone-24 5.96ns ± 1% 4.66ns ± 0% -21.85% (p=0.000 n=9+9) EncodeMVCCValue/header=empty/value=short-24 5.93ns ± 0% 4.66ns ± 0% -21.40% (p=0.000 n=9+9) EncodeMVCCValue/header=empty/value=long-24 5.92ns ± 0% 4.66ns ± 0% -21.31% (p=0.000 n=10+10) EncodeMVCCValue/header=local_walltime/value=tombstone-24 51.9ns ± 1% 49.5ns ± 1% -4.81% (p=0.000 n=9+10) EncodeMVCCValue/header=local_walltime/value=short-24 54.2ns ± 1% 52.5ns ± 1% -3.25% (p=0.000 n=10+10) EncodeMVCCValue/header=local_walltime/value=long-24 1.34µs ± 2% 1.36µs ± 1% +1.69% (p=0.001 n=10+9) EncodeMVCCValue/header=local_walltime+logical/value=tombstone-24 56.3ns ± 0% 53.3ns ± 1% -5.40% (p=0.000 n=10+10) EncodeMVCCValue/header=local_walltime+logical/value=short-24 58.8ns ± 0% 56.3ns ± 2% -4.18% (p=0.000 n=10+9) EncodeMVCCValue/header=local_walltime+logical/value=long-24 1.36µs ± 3% 1.36µs ± 1% ~ (p=0.269 n=10+9) ``` Release note: None --- pkg/storage/enginepb/mvcc3.go | 9 +++++++ pkg/storage/enginepb/mvcc3.proto | 2 ++ pkg/storage/mvcc_value.go | 44 ++++++++------------------------ pkg/storage/mvcc_value_test.go | 1 + 4 files changed, 22 insertions(+), 34 deletions(-) diff --git a/pkg/storage/enginepb/mvcc3.go b/pkg/storage/enginepb/mvcc3.go index 1cb6924095d5..4db670f1699b 100644 --- a/pkg/storage/enginepb/mvcc3.go +++ b/pkg/storage/enginepb/mvcc3.go @@ -51,3 +51,12 @@ func (op *MVCCLogicalOp) MustSetValue(value interface{}) { panic(errors.AssertionFailedf("%T excludes %T", op, value)) } } + +// IsEmpty returns true if the header is empty. +// gcassert:inline +func (h MVCCValueHeader) IsEmpty() bool { + // NB: We don't use a struct comparison like h == MVCCValueHeader{} due to a + // Go 1.19 performance regression, see: + // https://github.com/cockroachdb/cockroach/issues/88818 + return h.LocalTimestamp.IsEmpty() +} diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index 693e306f00ad..2ac77614ca7d 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -142,6 +142,8 @@ message IgnoredSeqNumRange { // MVCCValueHeader holds MVCC-level metadata for a versioned value. // Used by storage.MVCCValue. +// +// NB: Make sure to update MVCCValueHeader.IsEmpty() when adding fields. message MVCCValueHeader { option (gogoproto.equal) = true; diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go index 82a847a40a0f..e540a5213ee8 100644 --- a/pkg/storage/mvcc_value.go +++ b/pkg/storage/mvcc_value.go @@ -146,42 +146,20 @@ var disableSimpleValueEncoding = util.ConstantWithMetamorphicTestBool( "mvcc-value-disable-simple-encoding", false) // DisableMetamorphicSimpleValueEncoding disables the disableSimpleValueEncoding -// metamorphic bool and emptyValueHeader value for the duration of a test, -// resetting it at the end. -func DisableMetamorphicSimpleValueEncoding(t *testing.T) { +// metamorphic bool for the duration of a test, resetting it at the end. +func DisableMetamorphicSimpleValueEncoding(t testing.TB) { t.Helper() if disableSimpleValueEncoding { disableSimpleValueEncoding = false - oldHeader := emptyValueHeader - emptyValueHeader = enginepb.MVCCValueHeader{} - t.Cleanup(func() { disableSimpleValueEncoding = true - emptyValueHeader = oldHeader }) } } -var emptyValueHeader = func() enginepb.MVCCValueHeader { - var h enginepb.MVCCValueHeader - // Hacky: we don't have room in the mid-stack inlining budget in either - // encodedMVCCValueSize or EncodeMVCCValue to add to the simple encoding - // condition (e.g. `&& !disableSimpleValueEncoding`). So to have the same - // effect, we replace the empty value header with a header we never expect - // to see. We never expect LocalTimestamp to be set to MaxClockTimestamp - // because if it was set to that value, LocalTimestampNeeded would never - // return true. - if disableSimpleValueEncoding { - h.LocalTimestamp = hlc.MaxClockTimestamp - } - return h -}() - // encodedMVCCValueSize returns the size of the MVCCValue when encoded. -// -//gcassert:inline func encodedMVCCValueSize(v MVCCValue) int { - if v.MVCCValueHeader == emptyValueHeader { + if v.MVCCValueHeader.IsEmpty() && !disableSimpleValueEncoding { return len(v.Value.RawBytes) } return extendedPreludeSize + v.MVCCValueHeader.Size() + len(v.Value.RawBytes) @@ -190,22 +168,20 @@ func encodedMVCCValueSize(v MVCCValue) int { // EncodeMVCCValue encodes an MVCCValue into its Pebble representation. See the // comment on MVCCValue for a description of the encoding scheme. // -//gcassert:inline +// TODO(erikgrinaker): This could mid-stack inline when we compared +// v.MVCCValueHeader == enginepb.MVCCValueHeader{} instead of IsEmpty(), but +// struct comparisons have a significant performance regression in Go 1.19 which +// negates the inlining gain. Reconsider this with Go 1.20. See: +// https://github.com/cockroachdb/cockroach/issues/88818 func EncodeMVCCValue(v MVCCValue) ([]byte, error) { - if v.MVCCValueHeader == emptyValueHeader { + if v.MVCCValueHeader.IsEmpty() && !disableSimpleValueEncoding { // Simple encoding. Use the roachpb.Value encoding directly with no // modification. No need to re-allocate or copy. return v.Value.RawBytes, nil } + // Extended encoding. Wrap the roachpb.Value encoding with a header containing // MVCC-level metadata. Requires a re-allocation and copy. - return encodeExtendedMVCCValue(v) -} - -// encodeExtendedMVCCValue implements the extended MVCCValue encoding. It is -// split from EncodeMVCCValue to allow that function to qualify for mid-stack -// inlining, which avoids a function call for the simple encoding. -func encodeExtendedMVCCValue(v MVCCValue) ([]byte, error) { headerLen := v.MVCCValueHeader.Size() headerSize := extendedPreludeSize + headerLen valueSize := headerSize + len(v.Value.RawBytes) diff --git a/pkg/storage/mvcc_value_test.go b/pkg/storage/mvcc_value_test.go index 95b763e9d3f5..28262ade08f8 100644 --- a/pkg/storage/mvcc_value_test.go +++ b/pkg/storage/mvcc_value_test.go @@ -183,6 +183,7 @@ var mvccValueBenchmarkConfigs = struct { } func BenchmarkEncodeMVCCValue(b *testing.B) { + DisableMetamorphicSimpleValueEncoding(b) cfg := mvccValueBenchmarkConfigs for hDesc, h := range cfg.headers { for vDesc, v := range cfg.values { From aec769d375473fad73af91b18be27b5f1b81975e Mon Sep 17 00:00:00 2001 From: Sean Barag Date: Tue, 27 Sep 2022 17:01:59 -0700 Subject: [PATCH 2/2] server: compress gzip-encoded HTTP responses HTTP requests with `Accept-encoding: gzip` result in a gzip-encoded response with `Content-encoding: gzip` as expected, but with a gzip body that was uncompressed. This resulted in responses that were slightly larger than their uncompressed versions due to the inclusion of gzip headers and trailers. Replace gzip_response_writer with github.com/NYTimes/gziphandler, and compress gzip-encoded HTTP responses. This increases response time for extremely large responses (e.g. 25MB uncompressed for /bundle.js) to approximately 500ms in exchange for a reduced transfer size (e.g. transferring 7MB compressed for /bundle.js). fixes https://github.com/cockroachdb/cockroach/issues/88934 Release note (performance improvement): HTTP requests with 'Accept-encoding: gzip' previously resulted in valid gzip-encoded but uncompressed responses. This resulted in inefficient HTTP transfer times, as far more bytes were transferred than necessary. Those responses are now properly compressed, resulting in smaller network responses. --- go.mod | 1 + go.sum | 1 + pkg/server/BUILD.bazel | 2 +- pkg/server/gzip_response_writer.go | 75 ------------------------------ pkg/server/server_http.go | 45 ++++-------------- pkg/server/server_test.go | 22 ++++++++- vendor | 2 +- 7 files changed, 34 insertions(+), 114 deletions(-) delete mode 100644 pkg/server/gzip_response_writer.go diff --git a/go.mod b/go.mod index 215005f0a80f..a3212bdbd2f4 100644 --- a/go.mod +++ b/go.mod @@ -14,6 +14,7 @@ require ( github.com/BurntSushi/toml v0.4.1 github.com/Masterminds/semver/v3 v3.1.1 github.com/MichaelTJones/walk v0.0.0-20161122175330-4748e29d5718 + github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46 github.com/PuerkitoBio/goquery v1.5.1 github.com/Shopify/sarama v1.29.0 github.com/VividCortex/ewma v1.1.1 diff --git a/go.sum b/go.sum index 5362bb363de7..84e4cf4c62f8 100644 --- a/go.sum +++ b/go.sum @@ -218,6 +218,7 @@ github.com/Microsoft/hcsshim v0.8.16/go.mod h1:o5/SZqmR7x9JNKsW3pu+nqHm0MF8vbA+V github.com/Microsoft/hcsshim v0.8.18/go.mod h1:+w2gRZ5ReXQhFOrvSQeNfhrYB/dg3oDwTOcER2fw4I4= github.com/Microsoft/hcsshim/test v0.0.0-20201218223536-d3e5debf77da/go.mod h1:5hlzMzRKMLyo42nCZ9oml8AdTlq/0cvIaBv6tK1RehU= github.com/Microsoft/hcsshim/test v0.0.0-20210227013316-43a75bb4edd3/go.mod h1:mw7qgWloBUl75W/gVH3cQszUg1+gUITj7D6NY7ywVnY= +github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46 h1:lsxEuwrXEAokXB9qhlbKWPpo3KMLZQ5WB5WLQRW1uq0= github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 139d1153480d..de295c9fcbe8 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -29,7 +29,6 @@ go_library( "external_storage_builder.go", "grpc_gateway.go", "grpc_server.go", - "gzip_response_writer.go", "import_ts.go", "index_usage_stats.go", "init.go", @@ -288,6 +287,7 @@ go_library( "@com_github_grpc_ecosystem_grpc_gateway//utilities:go_default_library", "@com_github_marusama_semaphore//:semaphore", "@com_github_nightlyone_lockfile//:lockfile", + "@com_github_nytimes_gziphandler//:gziphandler", "@in_gopkg_yaml_v2//:yaml_v2", "@io_etcd_go_etcd_raft_v3//:raft", "@org_golang_google_grpc//:go_default_library", diff --git a/pkg/server/gzip_response_writer.go b/pkg/server/gzip_response_writer.go deleted file mode 100644 index 3052a2a2bf9d..000000000000 --- a/pkg/server/gzip_response_writer.go +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2022 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 server - -import ( - "compress/gzip" - "net/http" - "sync" -) - -// TODO(benesch): Use https://github.com/NYTimes/gziphandler instead. -// gzipResponseWriter reinvents the wheel and is not as robust. -type gzipResponseWriter struct { - gz gzip.Writer - http.ResponseWriter -} - -// Allocation pool for gzipResponseWriters. -var gzipResponseWriterPool sync.Pool - -func newGzipResponseWriter(rw http.ResponseWriter) *gzipResponseWriter { - var w *gzipResponseWriter - if wI := gzipResponseWriterPool.Get(); wI == nil { - w = new(gzipResponseWriter) - } else { - w = wI.(*gzipResponseWriter) - } - w.Reset(rw) - return w -} - -func (w *gzipResponseWriter) Reset(rw http.ResponseWriter) { - w.gz.Reset(rw) - w.ResponseWriter = rw -} - -func (w *gzipResponseWriter) Write(b []byte) (int, error) { - // The underlying http.ResponseWriter can't sniff gzipped data properly, so we - // do our own sniffing on the uncompressed data. - if w.Header().Get("Content-Type") == "" { - w.Header().Set("Content-Type", http.DetectContentType(b)) - } - return w.gz.Write(b) -} - -// Flush implements http.Flusher as required by grpc-gateway for clients -// which access streaming endpoints (as exercised by the acceptance tests -// at time of writing). -func (w *gzipResponseWriter) Flush() { - // If Flush returns an error, we'll see it on the next call to Write or - // Close as well, so we can ignore it here. - if err := w.gz.Flush(); err == nil { - // Flush the wrapped ResponseWriter as well, if possible. - if f, ok := w.ResponseWriter.(http.Flusher); ok { - f.Flush() - } - } -} - -// Close implements the io.Closer interface. It is not safe to use the -// writer after calling Close. -func (w *gzipResponseWriter) Close() error { - err := w.gz.Close() - w.Reset(nil) // release ResponseWriter reference. - gzipResponseWriterPool.Put(w) - return err -} diff --git a/pkg/server/server_http.go b/pkg/server/server_http.go index 1914250ffa78..1faba919e336 100644 --- a/pkg/server/server_http.go +++ b/pkg/server/server_http.go @@ -14,8 +14,8 @@ import ( "context" "crypto/tls" "net/http" - "strings" + "github.com/NYTimes/gziphandler" "github.com/cockroachdb/cmux" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/debug" @@ -23,18 +23,18 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/ts" "github.com/cockroachdb/cockroach/pkg/ui" - "github.com/cockroachdb/cockroach/pkg/util/httputil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/cockroach/pkg/util/stop" - "github.com/cockroachdb/errors" "google.golang.org/grpc/metadata" ) type httpServer struct { - cfg BaseConfig - mux http.ServeMux + cfg BaseConfig + mux http.ServeMux + // gzMux is an HTTP handler that gzip-compresses mux. + gzMux http.Handler proxy *nodeProxy } @@ -44,7 +44,7 @@ func newHTTPServer( parseNodeID ParseNodeIDFn, getNodeIDHTTPAddress GetNodeIDHTTPAddressFn, ) *httpServer { - return &httpServer{ + server := &httpServer{ cfg: cfg, proxy: &nodeProxy{ scheme: cfg.HTTPRequestScheme(), @@ -53,6 +53,8 @@ func newHTTPServer( rpcContext: rpcContext, }, } + server.gzMux = gziphandler.GzipHandler(http.HandlerFunc(server.mux.ServeHTTP)) + return server } // HSTSEnabled is a boolean that enables HSTS headers on the HTTP @@ -271,35 +273,6 @@ func (s *httpServer) start( }) } -// gzipHandler intercepts HTTP Requests and will gzip the response if -// the request contains the `Accept-Encoding: gzip` header. Requests -// are then delegated to the server mux. -func (s *httpServer) gzipHandler(w http.ResponseWriter, r *http.Request) { - ae := r.Header.Get(httputil.AcceptEncodingHeader) - switch { - case strings.Contains(ae, httputil.GzipEncoding): - w.Header().Set(httputil.ContentEncodingHeader, httputil.GzipEncoding) - gzw := newGzipResponseWriter(w) - defer func() { - // Certain requests must not have a body, yet closing the gzip writer will - // attempt to write the gzip header. Avoid logging a warning in this case. - // This is notably triggered by: - // - // curl -H 'Accept-Encoding: gzip' \ - // -H 'If-Modified-Since: Thu, 29 Mar 2018 22:36:32 GMT' \ - // -v http://localhost:8080/favicon.ico > /dev/null - // - // which results in a 304 Not Modified. - if err := gzw.Close(); err != nil && !errors.Is(err, http.ErrBodyNotAllowed) { - ctx := s.cfg.AmbientCtx.AnnotateCtx(r.Context()) - log.Ops.Warningf(ctx, "error closing gzip response writer: %v", err) - } - }() - w = gzw - } - s.mux.ServeHTTP(w, r) -} - // baseHandler is the top-level HTTP handler for all HTTP traffic, before // authentication and authorization. // @@ -328,5 +301,5 @@ func (s *httpServer) baseHandler(w http.ResponseWriter, r *http.Request) { } }() - s.proxy.nodeProxyHandler(w, r, s.gzipHandler) + s.proxy.nodeProxyHandler(w, r, s.gzMux.ServeHTTP) } diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 1ab96b98157b..ebfef420aa26 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -11,6 +11,7 @@ package server import ( + "bytes" "compress/gzip" "context" "fmt" @@ -343,6 +344,9 @@ func TestAcceptEncoding(t *testing.T) { t.Fatal(err) } + var uncompressedSize int64 + var compressedSize int64 + testData := []struct { acceptEncoding string newReader func(io.Reader) io.Reader @@ -379,13 +383,29 @@ func TestAcceptEncoding(t *testing.T) { if ce := resp.Header.Get(httputil.ContentEncodingHeader); ce != d.acceptEncoding { t.Fatalf("unexpected content encoding: '%s' != '%s'", ce, d.acceptEncoding) } - r := d.newReader(resp.Body) + + // Measure and stash resposne body length for later comparison + rawBytes, err := io.ReadAll(resp.Body) + require.NoError(t, err) + rawBodyLength := int64(len(rawBytes)) + if d.acceptEncoding == "" { + uncompressedSize = rawBodyLength + } else { + compressedSize = rawBodyLength + } + + r := d.newReader(bytes.NewReader(rawBytes)) var data serverpb.JSONResponse if err := jsonpb.Unmarshal(r, &data); err != nil { t.Error(err) } }() } + + // Ensure compressed responses are smaller than uncompressed ones when the + // uncompressed body would be larger than one MTU. + require.Greater(t, uncompressedSize, int64(1400), "gzip compression testing requires a response body > 1400 bytes (one MTU). Please update the test response.") + require.Less(t, compressedSize, uncompressedSize, "Compressed response body must be smaller than uncompressed response body") } func TestListenerFileCreation(t *testing.T) { diff --git a/vendor b/vendor index 552a42c5cadb..4bf63eb28c7a 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 552a42c5cadbf822f754b950808608cbdd6b3335 +Subproject commit 4bf63eb28c7a8b1e80826e959c0e6a93634f0204