diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index ec32f20fbdca..10fa5d7b8d44 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -31,7 +31,7 @@
kv.raft_log.synchronize | boolean | true | set to true to synchronize on Raft log writes to persistent storage ('false' risks data loss) |
kv.range.backpressure_range_size_multiplier | float | 2 | 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 |
kv.range_descriptor_cache.size | integer | 1000000 | maximum number of entries in the range descriptor and leaseholder caches |
-kv.range_merge.queue_enabled | boolean | true | whether the automatic merge queue is enabled |
+kv.range_merge.queue_enabled | boolean | false | whether the automatic merge queue is enabled |
kv.rangefeed.enabled | boolean | false | if set, rangefeed registration is enabled |
kv.snapshot_rebalance.max_rate | byte size | 2.0 MiB | the rate limit (bytes/sec) to use for rebalance snapshots |
kv.snapshot_recovery.max_rate | byte size | 8.0 MiB | the rate limit (bytes/sec) to use for recovery snapshots |
diff --git a/pkg/server/servemux_test.go b/pkg/server/servemux_test.go
new file mode 100644
index 000000000000..71fdee276f65
--- /dev/null
+++ b/pkg/server/servemux_test.go
@@ -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()
+}
diff --git a/pkg/server/server.go b/pkg/server/server.go
index 26e29cd876c9..9e6765eec20d 100644
--- a/pkg/server/server.go
+++ b/pkg/server/server.go
@@ -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"
)
@@ -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
@@ -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,
diff --git a/pkg/storage/merge_queue.go b/pkg/storage/merge_queue.go
index ef97073ebe5a..8ab8b91ac83a 100644
--- a/pkg/storage/merge_queue.go
+++ b/pkg/storage/merge_queue.go
@@ -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