Skip to content

Commit

Permalink
Merge #29230 #29236
Browse files Browse the repository at this point in the history
29230: server: deflake TestRapidRestarts r=benesch a=petermattis

In go1.10 and earlier, it was not safe to call `http.ServeMux.ServeHTTP`
concurrently with `http.ServeMux.Handle`. (This is fixed in go1.11). In
the interim, provide our own safeServeMux wrapper that provides proper
locking.

Fixes #29227

Release note: None

29236: Revert "storage: enable the merge queue by default" r=tschottdorf a=benesch

This reverts commit 98ca1d0. The merge
queue will be reenabled once flaky tests are fixed.

To reviewers: I'd much prefer to merge #29235. But if that gets stuck in code review or the flakiness reaches a breaking point, feel free to merge this instead.

Co-authored-by: Peter Mattis <[email protected]>
Co-authored-by: Nikhil Benesch <[email protected]>
  • Loading branch information
3 people committed Aug 29, 2018
3 parents de31594 + f932530 + a1cc4c5 commit e6e10de
Show file tree
Hide file tree
Showing 4 changed files with 92 additions and 4 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
<tr><td><code>kv.raft_log.synchronize</code></td><td>boolean</td><td><code>true</code></td><td>set to true to synchronize on Raft log writes to persistent storage ('false' risks data loss)</td></tr>
<tr><td><code>kv.range.backpressure_range_size_multiplier</code></td><td>float</td><td><code>2</code></td><td>multiple of range_max_bytes that a range is allowed to grow to without splitting before writes to that range are blocked, or 0 to disable</td></tr>
<tr><td><code>kv.range_descriptor_cache.size</code></td><td>integer</td><td><code>1000000</code></td><td>maximum number of entries in the range descriptor and leaseholder caches</td></tr>
<tr><td><code>kv.range_merge.queue_enabled</code></td><td>boolean</td><td><code>true</code></td><td>whether the automatic merge queue is enabled</td></tr>
<tr><td><code>kv.range_merge.queue_enabled</code></td><td>boolean</td><td><code>false</code></td><td>whether the automatic merge queue is enabled</td></tr>
<tr><td><code>kv.rangefeed.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if set, rangefeed registration is enabled</td></tr>
<tr><td><code>kv.snapshot_rebalance.max_rate</code></td><td>byte size</td><td><code>2.0 MiB</code></td><td>the rate limit (bytes/sec) to use for rebalance snapshots</td></tr>
<tr><td><code>kv.snapshot_recovery.max_rate</code></td><td>byte size</td><td><code>8.0 MiB</code></td><td>the rate limit (bytes/sec) to use for recovery snapshots</td></tr>
Expand Down
66 changes: 66 additions & 0 deletions pkg/server/servemux_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package server

import (
"fmt"
"net/http"
"net/http/httptest"
"net/url"
"sync"
"testing"
"time"

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

func TestServeMuxConcurrency(t *testing.T) {
defer leaktest.AfterTest(t)()

const duration = 20 * time.Millisecond
start := timeutil.Now()

// TODO(peter): This test reliably fails using http.ServeMux with a
// "concurrent map read and write error" on go1.10. The bug in http.ServeMux
// is fixed in go1.11.
var mux safeServeMux
var wg sync.WaitGroup
wg.Add(2)

go func() {
defer wg.Done()
f := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {})
for i := 1; timeutil.Since(start) < duration; i++ {
mux.Handle(fmt.Sprintf("/%d", i), f)
}
}()

go func() {
defer wg.Done()
for i := 1; timeutil.Since(start) < duration; i++ {
r := &http.Request{
Method: "GET",
URL: &url.URL{
Path: "/",
},
}
w := httptest.NewRecorder()
mux.ServeHTTP(w, r)
}
}()

wg.Wait()
}
26 changes: 24 additions & 2 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)
Expand Down Expand Up @@ -123,13 +124,35 @@ var (
)
)

// TODO(peter): Until go1.11, ServeMux.ServeHTTP was not safe to call
// concurrently with ServeMux.Handle. So we provide our own wrapper with proper
// locking. Slightly less efficient because it locks unnecessarily, but
// safe. See TestServeMuxConcurrency. Should remove once we've upgraded to
// go1.11.
type safeServeMux struct {
mu syncutil.RWMutex
mux http.ServeMux
}

func (mux *safeServeMux) Handle(pattern string, handler http.Handler) {
mux.mu.Lock()
mux.mux.Handle(pattern, handler)
mux.mu.Unlock()
}

func (mux *safeServeMux) ServeHTTP(w http.ResponseWriter, r *http.Request) {
mux.mu.RLock()
mux.mux.ServeHTTP(w, r)
mux.mu.RUnlock()
}

// Server is the cockroach server node.
type Server struct {
nodeIDContainer base.NodeIDContainer

cfg Config
st *cluster.Settings
mux *http.ServeMux
mux safeServeMux
clock *hlc.Clock
rpcContext *rpc.Context
grpc *grpc.Server
Expand Down Expand Up @@ -182,7 +205,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
clock := hlc.NewClock(hlc.UnixNano, time.Duration(cfg.MaxOffset))
s := &Server{
st: st,
mux: http.NewServeMux(),
clock: clock,
stopper: stopper,
cfg: cfg,
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/merge_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ const (
var MergeQueueEnabled = settings.RegisterBoolSetting(
"kv.range_merge.queue_enabled",
"whether the automatic merge queue is enabled",
true,
false,
)

// MergeQueueInterval is a setting that controls how often the merge queue waits
Expand Down

0 comments on commit e6e10de

Please sign in to comment.