Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
46829: sql: deal with retriable errors when using a new txn r=ajwerner a=ajwerner

In #46588 a bug was introduced when a retriable error was encountered while
using a new transaction for preparing. Prior to that commit, all error were
treated as not retriable. This was sort of a bummer. Retriable errors can
occur due to read within uncertainty. Before this PR, those retriable errors
would make their way to the client. Now we'll handle those retry errors
internally underneath `connExecutor.prepare`

Fixes #43251

Release note: None

46832: sqlmigrations: prevent schema change noise upon cluster creation r=lucy-zhang a=knz

First commit from  #46829.
Informs #46757.

The system.comments is now created without write permission to
public. No need to re-do that change on every new cluster.

Release note: None

46854: ui: (fix) Tooltip component styling and props r=dhartunian a=koorosh

This fix is rework of previous changes related to PR: #46557

Prior, to customize tooltip width for two particular cases,
changes were made in shared component and affected all tooltips
across project (tooltip width was set to 500px). 
For example, tooltips for Diagnostics badges were 500px wide:

<img width="1213" alt="Screenshot 2020-04-01 at 17 58 55" src="https://user-images.githubusercontent.com/3106437/78152553-a3068b00-7442-11ea-9575-f582cacc5ca4.png">

Instead of this, current changes keep component styles without
local changes and extend them with specific classes (via `overlayClassName`
prop). It allows to apply changes in specific places (Statements and Jobs
tables).

<img width="706" alt="Screenshot 2020-04-01 at 17 52 20" src="https://user-images.githubusercontent.com/3106437/78151930-cda41400-7441-11ea-8684-1eacd68f6934.png">

Next fix: the order of destructing props in `components/tooltip/tooltip.tsx`.
`{...props}` supplied as a last prop to `<AntTooltip />` component and it
overrides all previous props which have to be preserved. To fix this, ...props
was moved as first prop.

And last fix: Tooltips for Diagnostics Status Badge was set to be visible always
and with some random conditions tooltips appeared and were displayed instantly.
To fix this, `visible` prop was removed to trigger tooltip visibility only on
mouse hover.
And to position Diagnostics Status Badge tooltip more elegantly - it is positioned
to `bottomLeft` side, because this badge is displayed in the last columns and there
is not enough place on the right side for tooltip.

<img width="1198" alt="Screenshot 2020-04-01 at 17 51 35" src="https://user-images.githubusercontent.com/3106437/78151950-d4cb2200-7441-11ea-9b6d-e04a7f0d246f.png">

Release note (bug fix): Tooltips for statement diagnostics were shown always
instead of only on hover

Release justification: bug fixes and low-risk updates to new functionality

46871: vendor: Bump pebble to 74d69792cb150369fb7a799be862524ad2b39d51 r=itsbilal a=itsbilal

Pulls in these changes:
 - *: use errors.Errorf in replace of fmt.Errorf
 - *: use github.com/cockroachdb/errors
 - vendor: add cockroachdb/errors and its dependencies
 - *: add FileNum type
 - db: delete orphaned temporary files in Open
 - version_set: Handle case where RocksDB doesn't bump minUnflushedLogNum
 - db: modify Options.DebugCheck to be a function
 - internal/manifest: Relax SeqNum overlap invariant in L0
 - internal/metamorphic: fix ingest_using_apply implementation
 - *: use Go 1.13 error wrapping
 - internal/metamorphic: randomize MaxConcurrentCompactions
 - internal/metamorphic: enable run comparison by default
 - internal/metamorphic: temporarily disable ingest_using_apply
 - db: document that Close may not be called twice
 - db: use require.* functions for test checks
 - *: use require.NoError everywhere
 - internal/errorfs: move to new package

Release note: None

Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Andrii Vorobiov <[email protected]>
Co-authored-by: Bilal Akhtar <[email protected]>
  • Loading branch information
5 people committed Apr 1, 2020
5 parents f169a87 + 52653e6 + de30021 + a2dde8d + 4ef2d9a commit 3b3c470
Show file tree
Hide file tree
Showing 15 changed files with 116 additions and 73 deletions.
4 changes: 2 additions & 2 deletions Gopkg.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

45 changes: 24 additions & 21 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,34 +164,32 @@ func (ex *connExecutor) prepare(
// Preparing needs a transaction because it needs to retrieve db/table
// descriptors for type checking. If we already have an open transaction for
// this planner, use it. Using the user's transaction here is critical for
// proper deadlock detection. At the time of writing it is the case that any
// proper deadlock detection. At the time of writing, it is the case that any
// data read on behalf of this transaction is not cached for use in other
// transactions. It's critical that this fact remain true but nothing really
// enforces it. If we create a new transaction (newTxn is true), we'll need to
// finish it before we return.
newTxn, txn := false, ex.state.mu.txn
if txn == nil || !txn.IsOpen() {
newTxn, txn = true, kv.NewTxn(ctx, ex.server.cfg.DB, ex.server.cfg.NodeID.Get())

var flags planFlags
prepare := func(ctx context.Context, txn *kv.Txn) (err error) {
ex.statsCollector.reset(&ex.server.sqlStats, ex.appStats, &ex.phaseTimes)
p := &ex.planner
ex.resetPlanner(ctx, p, txn, ex.server.cfg.Clock.PhysicalTime() /* stmtTS */)
p.stmt = &stmt
p.semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations)
flags, err = ex.populatePrepared(ctx, txn, placeholderHints, p)
return err
}

ex.statsCollector.reset(&ex.server.sqlStats, ex.appStats, &ex.phaseTimes)
p := &ex.planner
ex.resetPlanner(ctx, p, txn, ex.server.cfg.Clock.PhysicalTime() /* stmtTS */)
p.stmt = &stmt
p.semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations)
flags, err := ex.populatePrepared(ctx, txn, placeholderHints, p)
if err != nil {
// NB: if this is not a new transaction then let the connExecutor state
// machine decide whether we should clean up intents; we may be restarting
// and want to leave them in place.
if newTxn {
txn.CleanupOnError(ctx, err)
if txn := ex.state.mu.txn; txn != nil && txn.IsOpen() {
// Use the existing transaction.
if err := prepare(ctx, txn); err != nil {
return nil, err
}
return nil, err
}
if newTxn {
// Clean up the newly created transaction if we made one.
if err := txn.CommitOrCleanup(ctx); err != nil {
} else {
// Use a new transaction. This will handle retriable errors here rather
// than bubbling them up to the connExecutor state machine.
if err := ex.server.cfg.DB.Txn(ctx, prepare); err != nil {
return nil, err
}
}
Expand All @@ -209,6 +207,11 @@ func (ex *connExecutor) prepare(
func (ex *connExecutor) populatePrepared(
ctx context.Context, txn *kv.Txn, placeholderHints tree.PlaceholderTypes, p *planner,
) (planFlags, error) {
if before := ex.server.cfg.TestingKnobs.BeforePrepare; before != nil {
if err := before(ctx, ex.planner.stmt.String(), txn); err != nil {
return 0, err
}
}
stmt := p.stmt
if err := p.semaCtx.Placeholders.Init(stmt.NumPlaceholders, placeholderHints); err != nil {
return 0, err
Expand Down
32 changes: 32 additions & 0 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/storagebase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -631,6 +632,37 @@ func TestPrepareInExplicitTransactionDoesNotDeadlock(t *testing.T) {
}
}

// TestRetriableErrorDuringPrepare ensures that when preparing and using a new
// transaction, retriable errors are handled properly and do not propagate to
// the user's transaction.
func TestRetriableErrorDuringPrepare(t *testing.T) {
defer leaktest.AfterTest(t)()
const uniqueString = "'a very unique string'"
var failed int64
const numToFail = 2 // only fail on the first two attempts
s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
SQLExecutor: &sql.ExecutorTestingKnobs{
BeforePrepare: func(ctx context.Context, stmt string, txn *kv.Txn) error {
if strings.Contains(stmt, uniqueString) && atomic.AddInt64(&failed, 1) <= numToFail {
return roachpb.NewTransactionRetryWithProtoRefreshError("boom",
txn.ID(), *txn.TestingCloneTxn())
}
return nil
},
},
},
})
defer s.Stopper().Stop(context.Background())

testDB := sqlutils.MakeSQLRunner(sqlDB)
testDB.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY)")

stmt, err := sqlDB.Prepare("SELECT " + uniqueString)
require.NoError(t, err)
defer func() { _ = stmt.Close() }()
}

// This test ensures that when in an explicit transaction and statement
// preparation uses the user's transaction, errors during those planning queries
// are handled correctly.
Expand Down
37 changes: 16 additions & 21 deletions pkg/sql/drop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,11 +231,10 @@ INSERT INTO t.kv VALUES ('c', 'e'), ('a', 'c'), ('b', 'd');
}

// Job still running, waiting for GC.
// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, 4, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
if err := jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: security.RootUser,
Description: "DROP DATABASE t CASCADE",
DescriptorIDs: sqlbase.IDs{
Expand Down Expand Up @@ -382,10 +381,9 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a');
tests.CheckKeyCount(t, kvDB, tableSpan, 6)
tests.CheckKeyCount(t, kvDB, table2Span, 6)

// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
const migrationJobOffset = 4
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
const migrationJobOffset = 0
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: security.RootUser,
Expand Down Expand Up @@ -559,10 +557,9 @@ func TestDropIndex(t *testing.T) {
tests.CheckKeyCount(t, kvDB, indexSpan, numRows)
tests.CheckKeyCount(t, kvDB, tableDesc.TableSpan(), 3*numRows)

// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
const migrationJobOffset = 4
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
const migrationJobOffset = 0
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset+1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: security.RootUser,
Expand Down Expand Up @@ -776,7 +773,7 @@ func TestDropTable(t *testing.T) {

// Job still running, waiting for GC.
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, 5, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
if err := jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: security.RootUser,
Description: `DROP TABLE t.public.kv`,
DescriptorIDs: sqlbase.IDs{
Expand Down Expand Up @@ -848,10 +845,9 @@ func TestDropTableDeleteData(t *testing.T) {
}
}

// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
const migrationJobOffset = 4
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
const migrationJobOffset = 0

// Data hasn't been GC-ed.
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
Expand Down Expand Up @@ -1133,12 +1129,11 @@ func TestDropDatabaseAfterDropTable(t *testing.T) {
}

// Job still running, waiting for draining names.
// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(
t, sqlRun, 5, jobspb.TypeSchemaChange, jobs.StatusSucceeded,
t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded,
jobs.Record{
Username: security.RootUser,
Description: "DROP TABLE t.public.kv",
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -652,6 +652,10 @@ type ExecutorTestingKnobs struct {
// statement has been executed.
StatementFilter StatementFilter

// BeforePrepare can be used to trap execution of SQL statement preparation.
// If a nil error is returned, planning continues as usual.
BeforePrepare func(ctx context.Context, stmt string, txn *kv.Txn) error

// BeforeExecute is called by the Executor before plan execution. It is useful
// for synchronizing statement execution.
BeforeExecute func(ctx context.Context, stmt string)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -625,7 +625,7 @@ system public web_sessions BASE TABLE
system public table_statistics BASE TABLE YES 1
system public locations BASE TABLE YES 1
system public role_members BASE TABLE YES 1
system public comments BASE TABLE YES 5
system public comments BASE TABLE YES 1
system public replication_constraint_stats BASE TABLE YES 1
system public replication_critical_localities BASE TABLE YES 1
system public replication_stats BASE TABLE YES 1
Expand Down
21 changes: 9 additions & 12 deletions pkg/sql/schema_changer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1843,10 +1843,9 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT8);
// State of jobs table
t.Skip("TODO(pbardea): The following fails due to causes seemingly unrelated to GC")
runner := sqlutils.SQLRunner{DB: sqlDB}
// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
const migrationJobOffset = 4
// TODO (lucy): This test API should use an offset starting from the
// most recent job instead.
const migrationJobOffset = 0
for i, tc := range testCases {
status := jobs.StatusSucceeded
if tc.errString != "" {
Expand Down Expand Up @@ -3989,10 +3988,9 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL REFERENCES t.pi (d) DE
// Ensure that the job is marked as succeeded.
sqlRun := sqlutils.MakeSQLRunner(sqlDB)

// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
schemaChangeJobOffset := 4
// TODO (lucy): This test API should use an offset starting from the
// most recent job instead.
schemaChangeJobOffset := 0
if err := jobutils.VerifySystemJob(t, sqlRun, schemaChangeJobOffset+2, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: security.RootUser,
Description: "TRUNCATE TABLE t.public.test",
Expand Down Expand Up @@ -5542,10 +5540,9 @@ INSERT INTO t.test (k, v) VALUES (1, 99), (2, 100);

sqlRun := sqlutils.MakeSQLRunner(sqlDB)
runBeforeConstraintValidation = func() error {
// TODO (lucy): The offset of +4 accounts for unrelated startup migrations.
// Maybe this test API should use an offset starting from the most recent job
// instead.
return jobutils.VerifyRunningSystemJob(t, sqlRun, 4, jobspb.TypeSchemaChange, sql.RunningStatusValidation, jobs.Record{
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
return jobutils.VerifyRunningSystemJob(t, sqlRun, 0, jobspb.TypeSchemaChange, sql.RunningStatusValidation, jobs.Record{
Username: security.RootUser,
Description: "ALTER TABLE t.public.test ADD COLUMN a INT8 AS (v - 1) STORED, ADD CHECK ((a < v) AND (a IS NOT NULL))",
DescriptorIDs: sqlbase.IDs{
Expand Down
5 changes: 3 additions & 2 deletions pkg/sqlmigrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -285,8 +285,9 @@ var backwardCompatibleMigrations = []migrationDescriptor{
},
{
// Introduced in v20.1.
name: "remove public permissions on system.comments",
workFn: depublicizeSystemComments,
name: "remove public permissions on system.comments",
includedInBootstrap: clusterversion.VersionByKey(clusterversion.VersionSchemaChangeJob),
workFn: depublicizeSystemComments,
},
{
// Introduced in v20.1.
Expand Down
2 changes: 0 additions & 2 deletions pkg/ui/src/components/tooltip/tooltip.styl
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,7 @@
@require '~src/components/core/index.styl'

.tooltip-overlay
max-width max-content
.ant-tooltip-content
width 500px
.ant-tooltip-inner
@extend $text--body
line-height $line-height--small
Expand Down
6 changes: 3 additions & 3 deletions pkg/ui/src/components/tooltip/tooltip.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,13 @@ export interface TooltipProps {
}

export function Tooltip(props: TooltipProps & AntTooltipProps) {
const { children, theme } = props;
const classes = cn("tooltip-overlay", `crl-tooltip--theme-${theme}`);
const { children, theme, overlayClassName } = props;
const classes = cn("tooltip-overlay", `crl-tooltip--theme-${theme}`, overlayClassName);
return (
<AntTooltip
{...props}
mouseEnterDelay={0.5}
overlayClassName={classes}
{...props}
>
{children}
</AntTooltip>
Expand Down
11 changes: 8 additions & 3 deletions pkg/ui/src/views/jobs/jobDescriptionCell.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,14 @@ export class JobDescriptionCell extends React.PureComponent<{ job: Job }> {
return (
<Link className={`${additionalStyle}`} to={`jobs/${String(job.id)}`}>
<div className="cl-table-link__tooltip">
<Tooltip arrowPointAtCenter placement="bottom" title={
<pre style={{whiteSpace: "pre-wrap"}} className="cl-table-link__description">{description}</pre>
}>
<Tooltip
arrowPointAtCenter
placement="bottom"
title={
<pre style={{whiteSpace: "pre-wrap"}} className="cl-table-link__description">{description}</pre>
}
overlayClassName="cl-table-link__statement-tooltip--fixed-width"
>
<div className="jobs-table__cell--description">{job.statement || job.description}</div>
</Tooltip>
</div>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,10 +99,9 @@ export function DiagnosticStatusBadge(props: OwnProps) {

return (
<Tooltip
visible={enableTooltip}
title={tooltipContent}
theme="blue"
placement="bottom"
placement="bottomLeft"
>
<div
className="diagnostic-status-badge__content"
Expand Down
5 changes: 5 additions & 0 deletions pkg/ui/src/views/statements/statements.styl
Original file line number Diff line number Diff line change
Expand Up @@ -415,3 +415,8 @@ $plan-node-attribute-key-color = #37a806 // light green
text-decoration underline
._text-bold
font-family RobotoMono-Bold

.cl-table-link__statement-tooltip--fixed-width
max-width max-content
.ant-tooltip-content
max-width 500px
10 changes: 7 additions & 3 deletions pkg/ui/src/views/statements/statementsTable.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,13 @@ function StatementLink(props: { statement: string, app: string, implicitTxn: boo
return (
<Link to={ `${base}/${encodeURIComponent(props.statement)}` }>
<div className="cl-table-link__tooltip">
<Tooltip placement="bottom" title={
<pre className="cl-table-link__description">{ getHighlightedText(props.statement, props.search) }</pre>
}>
<Tooltip
placement="bottom"
title={<pre className="cl-table-link__description">
{ getHighlightedText(props.statement, props.search) }
</pre>}
overlayClassName="cl-table-link__statement-tooltip--fixed-width"
>
<div className="cl-table-link__tooltip-hover-area">
{ getHighlightedText(shortStatement(summary, props.statement), props.search, true) }
</div>
Expand Down
2 changes: 1 addition & 1 deletion vendor
Submodule vendor updated 46 files
+9 −9 github.com/cockroachdb/pebble/batch.go
+23 −31 github.com/cockroachdb/pebble/compaction.go
+4 −4 github.com/cockroachdb/pebble/compaction_iter.go
+13 −13 github.com/cockroachdb/pebble/db.go
+17 −20 github.com/cockroachdb/pebble/event.go
+7 −3 github.com/cockroachdb/pebble/filenames.go
+1 −1 github.com/cockroachdb/pebble/flushable.go
+6 −0 github.com/cockroachdb/pebble/go.mod
+15 −0 github.com/cockroachdb/pebble/go.sum
+11 −13 github.com/cockroachdb/pebble/ingest.go
+2 −1 github.com/cockroachdb/pebble/internal/arenaskl/arena.go
+1 −1 github.com/cockroachdb/pebble/internal/arenaskl/skl.go
+1 −3 github.com/cockroachdb/pebble/internal/base/error.go
+41 −17 github.com/cockroachdb/pebble/internal/base/filenames.go
+8 −17 github.com/cockroachdb/pebble/internal/batchskl/skl.go
+11 −10 github.com/cockroachdb/pebble/internal/cache/clockpro.go
+1 −1 github.com/cockroachdb/pebble/internal/cache/robin_hood.go
+41 −34 github.com/cockroachdb/pebble/internal/manifest/version.go
+39 −29 github.com/cockroachdb/pebble/internal/manifest/version_edit.go
+3 −1 github.com/cockroachdb/pebble/internal/private/sstable.go
+7 −4 github.com/cockroachdb/pebble/internal/rate/rate.go
+5 −6 github.com/cockroachdb/pebble/internal/record/log_writer.go
+3 −2 github.com/cockroachdb/pebble/internal/record/record.go
+6 −6 github.com/cockroachdb/pebble/iterator.go
+15 −14 github.com/cockroachdb/pebble/level_checker.go
+10 −9 github.com/cockroachdb/pebble/log_recycler.go
+5 −4 github.com/cockroachdb/pebble/mem_table.go
+1 −1 github.com/cockroachdb/pebble/merging_iter.go
+18 −11 github.com/cockroachdb/pebble/open.go
+27 −18 github.com/cockroachdb/pebble/options.go
+2 −2 github.com/cockroachdb/pebble/read_state.go
+1 −1 github.com/cockroachdb/pebble/sstable/block.go
+2 −1 github.com/cockroachdb/pebble/sstable/raw_block.go
+18 −18 github.com/cockroachdb/pebble/sstable/reader.go
+7 −8 github.com/cockroachdb/pebble/sstable/table.go
+7 −7 github.com/cockroachdb/pebble/sstable/writer.go
+10 −11 github.com/cockroachdb/pebble/table_cache.go
+5 −4 github.com/cockroachdb/pebble/tool/db.go
+16 −16 github.com/cockroachdb/pebble/tool/find.go
+8 −8 github.com/cockroachdb/pebble/tool/lsm.go
+2 −2 github.com/cockroachdb/pebble/tool/make_incorrect_manifests.go
+6 −6 github.com/cockroachdb/pebble/tool/manifest.go
+2 −1 github.com/cockroachdb/pebble/tool/util.go
+35 −28 github.com/cockroachdb/pebble/version_set.go
+4 −3 github.com/cockroachdb/pebble/vfs/file_lock_generic.go
+2 −1 github.com/cockroachdb/pebble/vfs/mem_fs.go

0 comments on commit 3b3c470

Please sign in to comment.