Skip to content

Commit

Permalink
Merge #67121 #67146
Browse files Browse the repository at this point in the history
67121: kv: remove EvalContext.Engine r=nvanbenschoten a=nvanbenschoten

This change removes the Engine method from EvalContext. Removing this is
important, as its existence appears to undermine #55461 and make #66485
difficult.

The first place where this was used was in EndTxn's evaluation function.
I don't see any reason for this. In fact, we had a TODO to fix this, which
we could have addressed years ago.

The second place where this was used was in RecomputeStats's evaluation
function. There, it was used for two reasons. First, it was used because
`storage.Batch` used to not provide a consistent view of data. They now
do. It was also used to evade spanset assertions, which this commit
addresses in a better way.

67146: c-deps: remove protobuf r=dt a=dt

Release note: none.

Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: David Taylor <[email protected]>
  • Loading branch information
3 people committed Jul 6, 2021
3 parents 3fce327 + d8ff91e + d4ea9e4 commit 8f1279d
Show file tree
Hide file tree
Showing 11 changed files with 27 additions and 75 deletions.
3 changes: 0 additions & 3 deletions .gitmodules
Original file line number Diff line number Diff line change
Expand Up @@ -4,9 +4,6 @@
[submodule "c-deps/jemalloc"]
path = c-deps/jemalloc
url = https://github.com/cockroachdb/jemalloc.git
[submodule "c-deps/protobuf"]
path = c-deps/protobuf
url = https://github.com/cockroachdb/protobuf.git
[submodule "pkg/ui/yarn-vendor"]
path = pkg/ui/yarn-vendor
url = https://github.com/cockroachdb/yarn-vendored
Expand Down
1 change: 0 additions & 1 deletion BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@ load("@bazel_gazelle//:def.bzl", "gazelle")
#
# gazelle:exclude _bazel
# gazelle:exclude c-deps/krb5
# gazelle:exclude c-deps/protobuf
# gazelle:exclude artifacts
# gazelle:exclude vendor
# gazelle:exclude .vendor.tmp.*
Expand Down
26 changes: 4 additions & 22 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -487,7 +487,6 @@ endif

C_DEPS_DIR := $(abspath c-deps)
JEMALLOC_SRC_DIR := $(C_DEPS_DIR)/jemalloc
PROTOBUF_SRC_DIR := $(C_DEPS_DIR)/protobuf
GEOS_SRC_DIR := $(C_DEPS_DIR)/geos
PROJ_SRC_DIR := $(C_DEPS_DIR)/proj
LIBEDIT_SRC_DIR := $(C_DEPS_DIR)/libedit
Expand All @@ -512,16 +511,13 @@ BUILD_DIR := $(shell cygpath -m $(BUILD_DIR))
endif

JEMALLOC_DIR := $(BUILD_DIR)/jemalloc
PROTOBUF_DIR := $(BUILD_DIR)/protobuf
GEOS_DIR := $(BUILD_DIR)/geos
PROJ_DIR := $(BUILD_DIR)/proj
LIBEDIT_DIR := $(BUILD_DIR)/libedit
LIBROACH_DIR := $(BUILD_DIR)/libroach$(if $(ENABLE_LIBROACH_ASSERTIONS),_assert)
KRB5_DIR := $(BUILD_DIR)/krb5
# Can't share with protobuf because protoc is always built for the host.

LIBJEMALLOC := $(JEMALLOC_DIR)/lib/libjemalloc.a
LIBPROTOBUF := $(PROTOBUF_DIR)/libprotobuf.a
LIBEDIT := $(LIBEDIT_DIR)/src/.libs/libedit.a
LIBROACH := $(LIBROACH_DIR)/libroach.a
LIBPROJ := $(PROJ_DIR)/lib/libproj$(if $(target-is-windows),_4_9).a
Expand All @@ -543,8 +539,8 @@ C_LIBS_COMMON = \
$(if $(target-is-windows),,$(LIBEDIT)) \
$(LIBPROJ) $(LIBROACH)
C_LIBS_SHORT = $(C_LIBS_COMMON)
C_LIBS_OSS = $(C_LIBS_COMMON) $(LIBPROTOBUF)
C_LIBS_CCL = $(C_LIBS_COMMON) $(LIBPROTOBUF)
C_LIBS_OSS = $(C_LIBS_COMMON)
C_LIBS_CCL = $(C_LIBS_COMMON)
C_LIBS_DYNAMIC = $(LIBGEOS)

# We only include krb5 on linux, non-musl builds.
Expand Down Expand Up @@ -600,7 +596,7 @@ $(BASE_CGO_FLAGS_FILES): Makefile build/defs.mk.sig | bin/.submodules-initialize
@echo 'package $(if $($(@D)-package),$($(@D)-package),$(notdir $(@D)))' >> $@
@echo >> $@
@echo '// #cgo CPPFLAGS: $(addprefix -I,$(JEMALLOC_DIR)/include $(KRB_CPPFLAGS))' >> $@
@echo '// #cgo LDFLAGS: $(addprefix -L,$(PROTOBUF_DIR) $(JEMALLOC_DIR)/lib $(LIBEDIT_DIR)/src/.libs $(LIBROACH_DIR) $(KRB_DIR) $(PROJ_DIR)/lib)' >> $@
@echo '// #cgo LDFLAGS: $(addprefix -L,$(JEMALLOC_DIR)/lib $(LIBEDIT_DIR)/src/.libs $(LIBROACH_DIR) $(KRB_DIR) $(PROJ_DIR)/lib)' >> $@
@echo 'import "C"' >> $@

vendor/github.com/knz/go-libedit/unix/zcgo_flags_extra.go: Makefile | bin/.submodules-initialized
Expand Down Expand Up @@ -658,14 +654,6 @@ $(KRB5_DIR)/Makefile: $(C_DEPS_DIR)/krb5-rebuild $(KRB5_SRC_DIR)/src/configure
@# We specify -fcommon to get around duplicate definition errors in recent gcc.
cd $(KRB5_DIR) && env -u CXXFLAGS CFLAGS="-fcommon" $(KRB5_SRC_DIR)/src/configure $(xconfigure-flags) --enable-static --disable-shared

$(PROTOBUF_DIR)/Makefile: $(C_DEPS_DIR)/protobuf-rebuild | bin/.submodules-initialized
rm -rf $(PROTOBUF_DIR)
mkdir -p $(PROTOBUF_DIR)
@# NOTE: If you change the CMake flags below, bump the version in
@# $(C_DEPS_DIR)/protobuf-rebuild. See above for rationale.
cd $(PROTOBUF_DIR) && cmake $(xcmake-flags) -Dprotobuf_WITH_ZLIB=OFF -Dprotobuf_BUILD_TESTS=OFF $(PROTOBUF_SRC_DIR)/cmake \
-DCMAKE_BUILD_TYPE=Release

$(GEOS_DIR)/Makefile: $(C_DEPS_DIR)/geos-rebuild | bin/.submodules-initialized
rm -rf $(GEOS_DIR)
mkdir -p $(GEOS_DIR)
Expand Down Expand Up @@ -735,9 +723,6 @@ $(LIBEDIT_DIR)/Makefile: $(C_DEPS_DIR)/libedit-rebuild $(LIBEDIT_SRC_DIR)/config
$(LIBJEMALLOC): $(JEMALLOC_DIR)/Makefile bin/uptodate .ALWAYS_REBUILD
@uptodate $@ $(JEMALLOC_SRC_DIR) || $(MAKE) --no-print-directory -C $(JEMALLOC_DIR) build_lib_static

$(LIBPROTOBUF): $(PROTOBUF_DIR)/Makefile bin/uptodate .ALWAYS_REBUILD
@uptodate $@ $(PROTOBUF_SRC_DIR) || $(MAKE) --no-print-directory -C $(PROTOBUF_DIR) libprotobuf

ifdef is-cross-compile
ifdef target-is-macos
geos_require_install_name_tool := 1
Expand Down Expand Up @@ -788,10 +773,9 @@ $(LIBKRB5): $(KRB5_DIR)/Makefile bin/uptodate .ALWAYS_REBUILD
@uptodate $@ $(KRB5_SRC_DIR)/src || $(MAKE) --no-print-directory -C $(KRB5_DIR)

# Convenient names for maintainers. Not used by other targets in the Makefile.
.PHONY: libjemalloc libprotobuf libgeos libproj libroach libkrb5
.PHONY: libjemalloc libgeos libproj libroach libkrb5
libedit: $(LIBEDIT)
libjemalloc: $(LIBJEMALLOC)
libprotobuf: $(LIBPROTOBUF)
libgeos: $(LIBGEOS)
libproj: $(LIBPROJ)
libroach: $(LIBROACH)
Expand Down Expand Up @@ -1684,7 +1668,6 @@ c-deps-fmt:
.PHONY: clean-c-deps
clean-c-deps:
rm -rf $(JEMALLOC_DIR)
rm -rf $(PROTOBUF_DIR)
rm -rf $(GEOS_DIR)
rm -rf $(PROJ_DIR)
rm -rf $(LIBROACH_DIR)
Expand All @@ -1693,7 +1676,6 @@ clean-c-deps:
.PHONY: unsafe-clean-c-deps
unsafe-clean-c-deps:
git -C $(JEMALLOC_SRC_DIR) clean -dxf
git -C $(PROTOBUF_SRC_DIR) clean -dxf
git -C $(GEOS_SRC_DIR) clean -dxf
git -C $(PROJ_SRC_DIR) clean -dxf
git -C $(LIBROACH_SRC_DIR) clean -dxf
Expand Down
3 changes: 0 additions & 3 deletions build/variables.mk
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,6 @@ define VALID_VARS
LIBGEOS
LIBJEMALLOC
LIBPROJ
LIBPROTOBUF
LIBROACH
LIBROACH_DIR
LIBROACH_SRC_DIR
Expand All @@ -117,8 +116,6 @@ define VALID_VARS
PROJ_DIR
PROJ_SRC_DIR
PROMETHEUS_PATH
PROTOBUF_DIR
PROTOBUF_SRC_DIR
PROTOBUF_TARGETS
PROTO_MAPPINGS
RACETIMEOUT
Expand Down
10 changes: 0 additions & 10 deletions c-deps/REPOSITORIES.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,6 @@ BUILD_ALL_CONTENT = """filegroup(name = "all", srcs = glob(["**"]), visibility =
# building those libraries require certain checked out repositories being
# placed relative to the source tree of the library itself.

# For c-deps/protobuf, we elide a checked in generated file. Already generated
# files are read-only in the bazel sandbox, so bazel is unable to regenerate
# the same files, which the build process requires it to do so.
BUILD_PROTOBUF_CONTENT = """filegroup(name = "all", srcs = glob(["**"], exclude=["src/google/protobuf/compiler/js/well_known_types_embed.cc"]), visibility = ["//visibility:public"])"""

# This is essentially the same above, we elide a generated file to avoid
# permission issues when building jemalloc within the bazel sandbox.
BUILD_JEMALLOC_CONTENT = """filegroup(name = "all", srcs = glob(["**"], exclude=["configure"]), visibility = ["//visibility:public"])"""
Expand All @@ -37,11 +32,6 @@ def c_deps():
path = "c-deps/geos",
build_file_content = BUILD_ALL_CONTENT,
)
native.new_local_repository(
name = "protobuf",
path = "c-deps/protobuf",
build_file_content = BUILD_PROTOBUF_CONTENT,
)
native.new_local_repository(
name = "jemalloc",
path = "c-deps/jemalloc",
Expand Down
1 change: 0 additions & 1 deletion c-deps/protobuf
Submodule protobuf deleted from e809d7
9 changes: 3 additions & 6 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -949,11 +949,8 @@ func splitTriggerHelper(
// - node two becomes the lease holder for [c,e). Its timestamp cache does
// not know about the read at 'd' which happened at the beginning.
// - node two can illegally propose a write to 'd' at a lower timestamp.
//
// TODO(tschottdorf): why would this use r.store.Engine() and not the
// batch? We do the same thing for other usages of the state loader.
sl := MakeStateLoader(rec)
leftLease, err := sl.LoadLease(ctx, rec.Engine())
leftLease, err := sl.LoadLease(ctx, batch)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load lease")
}
Expand All @@ -970,7 +967,7 @@ func splitTriggerHelper(
}
rightLease := leftLease
rightLease.Replica = replica
gcThreshold, err := sl.LoadGCThreshold(ctx, rec.Engine())
gcThreshold, err := sl.LoadGCThreshold(ctx, batch)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold")
}
Expand Down Expand Up @@ -1001,7 +998,7 @@ func splitTriggerHelper(
truncStateType = stateloader.TruncatedStateLegacyReplicated
}

replicaVersion, err := sl.LoadVersion(ctx, rec.Engine())
replicaVersion, err := sl.LoadVersion(ctx, batch)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold")
}
Expand Down
25 changes: 7 additions & 18 deletions pkg/kv/kvserver/batcheval/cmd_recompute_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ func init() {
}

func declareKeysRecomputeStats(
rs ImmutableRangeState, _ roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
// We don't declare any user key in the range. This is OK since all we're doing is computing a
// stats delta, and applying this delta commutes with other operations on the same key space.
Expand All @@ -53,7 +53,7 @@ func declareKeysRecomputeStats(
// RecomputeStats recomputes the MVCCStats stored for this range and adjust them accordingly,
// returning the MVCCStats delta obtained in the process.
func RecomputeStats(
ctx context.Context, _ storage.Reader, cArgs CommandArgs, resp roachpb.Response,
ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response,
) (result.Result, error) {
desc := cArgs.EvalCtx.Desc()
args := cArgs.Args.(*roachpb.RecomputeStatsRequest)
Expand All @@ -64,27 +64,16 @@ func RecomputeStats(

args = nil // avoid accidental use below

// Open a snapshot from which we will read everything (including the
// MVCCStats). This is necessary because a batch does not provide us
// with a consistent view of the data -- reading from the batch, we
// could see skew between the stats recomputation and the MVCCStats
// we read from the range state if concurrent writes are inflight[1].
//
// Note that in doing so, we also circumvent the assertions (present in both
// the EvalContext and the batch in some builds) which check that all reads
// were previously declared. See the comment in `declareKeysRecomputeStats`
// for details on this.
//
// [1]: see engine.TestBatchReadLaterWrite.
snap := cArgs.EvalCtx.Engine().NewSnapshot()
defer snap.Close()
// Disable the assertions which check that all reads were previously declared.
// See the comment in `declareKeysRecomputeStats` for details on this.
reader = spanset.DisableReaderAssertions(reader)

actualMS, err := rditer.ComputeStatsForRange(desc, snap, cArgs.Header.Timestamp.WallTime)
actualMS, err := rditer.ComputeStatsForRange(desc, reader, cArgs.Header.Timestamp.WallTime)
if err != nil {
return result.Result{}, err
}

currentStats, err := MakeStateLoader(cArgs.EvalCtx).LoadMVCCStats(ctx, snap)
currentStats, err := MakeStateLoader(cArgs.EvalCtx).LoadMVCCStats(ctx, reader)
if err != nil {
return result.Result{}, err
}
Expand Down
5 changes: 0 additions & 5 deletions pkg/kv/kvserver/batcheval/eval_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -50,7 +49,6 @@ type EvalContext interface {
ClusterSettings() *cluster.Settings
EvalKnobs() kvserverbase.BatchEvalTestingKnobs

Engine() storage.Engine
Clock() *hlc.Clock
AbortSpan() *abortspan.AbortSpan
GetConcurrencyManager() concurrency.Manager
Expand Down Expand Up @@ -170,9 +168,6 @@ func (m *mockEvalCtxImpl) ClusterSettings() *cluster.Settings {
func (m *mockEvalCtxImpl) EvalKnobs() kvserverbase.BatchEvalTestingKnobs {
return kvserverbase.BatchEvalTestingKnobs{}
}
func (m *mockEvalCtxImpl) Engine() storage.Engine {
panic("unimplemented")
}
func (m *mockEvalCtxImpl) Clock() *hlc.Clock {
return m.MockEvalCtx.Clock
}
Expand Down
6 changes: 0 additions & 6 deletions pkg/kv/kvserver/replica_eval_context_span.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -86,11 +85,6 @@ func (rec *SpanSetReplicaEvalContext) GetNodeLocality() roachpb.Locality {
return rec.i.GetNodeLocality()
}

// Engine returns the engine.
func (rec *SpanSetReplicaEvalContext) Engine() storage.Engine {
return rec.i.Engine()
}

// GetFirstIndex returns the first index.
func (rec *SpanSetReplicaEvalContext) GetFirstIndex() (uint64, error) {
return rec.i.GetFirstIndex()
Expand Down
13 changes: 13 additions & 0 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -736,3 +736,16 @@ func NewBatchAt(b storage.Batch, spans *SpanSet, ts hlc.Timestamp) storage.Batch
ts: ts,
}
}

// DisableReaderAssertions unwraps any storage.Reader implementations that may
// assert access against a given SpanSet.
func DisableReaderAssertions(reader storage.Reader) storage.Reader {
switch v := reader.(type) {
case ReadWriter:
return DisableReaderAssertions(v.r)
case *spanSetBatch:
return DisableReaderAssertions(v.r)
default:
return reader
}
}

0 comments on commit 8f1279d

Please sign in to comment.