Skip to content

Commit

Permalink
Merge #88950 #88989
Browse files Browse the repository at this point in the history
88950: server: compress gzip-encoded HTTP responses r=knz a=sjbarag

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 #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.

88989: storage: avoid `EncodeMVCCValue` struct comparison r=erikgrinaker a=erikgrinaker

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)
```

Resolves #88818.

Release note: None

Co-authored-by: Sean Barag <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
3 people committed Oct 3, 2022
3 parents 9794be4 + aec769d + 7b20564 commit d36efd1
Show file tree
Hide file tree
Showing 11 changed files with 56 additions and 148 deletions.
1 change: 1 addition & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -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=
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
75 changes: 0 additions & 75 deletions pkg/server/gzip_response_writer.go

This file was deleted.

45 changes: 9 additions & 36 deletions pkg/server/server_http.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,27 +14,27 @@ 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"
"github.com/cockroachdb/cockroach/pkg/server/status"
"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
}

Expand All @@ -44,7 +44,7 @@ func newHTTPServer(
parseNodeID ParseNodeIDFn,
getNodeIDHTTPAddress GetNodeIDHTTPAddressFn,
) *httpServer {
return &httpServer{
server := &httpServer{
cfg: cfg,
proxy: &nodeProxy{
scheme: cfg.HTTPRequestScheme(),
Expand All @@ -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
Expand Down Expand Up @@ -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.
//
Expand Down Expand Up @@ -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)
}
22 changes: 21 additions & 1 deletion pkg/server/server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package server

import (
"bytes"
"compress/gzip"
"context"
"fmt"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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) {
Expand Down
9 changes: 9 additions & 0 deletions pkg/storage/enginepb/mvcc3.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}
2 changes: 2 additions & 0 deletions pkg/storage/enginepb/mvcc3.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down
44 changes: 10 additions & 34 deletions pkg/storage/mvcc_value.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/mvcc_value_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down

0 comments on commit d36efd1

Please sign in to comment.