Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

sql: do add/drop column backfills via distsql #14331

Merged
merged 6 commits into from
Mar 30, 2017

Conversation

jordanlewis
Copy link
Member

@jordanlewis jordanlewis commented Mar 22, 2017

  • Move rowwriter and makeDefaultExprs into sqlbase so they can be used from the distsql package.
  • Replace the current implementation of backfilling an add/drop column schema change with one that uses distsql the way that the index backfiller works.

This change is Reviewable

@jordanlewis
Copy link
Member Author

This is out for early review. The implementation currently passes logic tests but does not pass TestRaceWithBackfill, so there is still some work to be done.

@vivekmenezes
Copy link
Contributor

Reviewed 5 of 5 files at r1, 13 of 13 files at r2, 1 of 1 files at r3, 2 of 2 files at r4.
Review status: 15 of 20 files reviewed at latest revision, 10 unresolved discussions, some commit checks failed.


pkg/sql/backfill.go, line 144 at r5 (raw file):

			switch t := m.Descriptor_.(type) {
			case *sqlbase.DescriptorMutation_Column:
				addedColumnDescs = append(addedColumnDescs, *t.Column)

we don't need addedColumnDescs anymore


pkg/sql/backfill.go, line 164 at r5 (raw file):

			switch t := m.Descriptor_.(type) {
			case *sqlbase.DescriptorMutation_Column:
				droppedColumnDescs = append(droppedColumnDescs, *t.Column)

needColumnBackfill = true

we use the same backfill mechanism to delete a column

get rid of droppedColumnDescs


pkg/sql/distsql_physical_planner.go, line 622 at r5 (raw file):

			Duration:    duration,
			ChunkSize:   chunkSize,
			OtherTables: fkDescs,

might as well create a generic distsqlrun.BackfillerSpec and only use the switch to set Type


pkg/sql/distsqlrun/columnbackfiller.go, line 36 at r5 (raw file):

	added        []sqlbase.ColumnDescriptor
	dropped      []sqlbase.ColumnDescriptor
	updateCols   []sqlbase.ColumnDescriptor

updateCols?


pkg/sql/distsqlrun/columnbackfiller.go, line 41 at r5 (raw file):

	nonNullViolationColumnName string

	// colIdxMap maps ColumnIDs to indices into desc.Columns and desc.Mutations.

and desc.Mutations that hold column descriptors.


pkg/sql/distsqlrun/columnbackfiller.go, line 89 at r5 (raw file):

		for _, m := range desc.Mutations {
			if ColumnMutationFilter(m) {
				switch m.Direction {

I think you meant to also append the mutation column to cols


pkg/sql/distsqlrun/columnbackfiller.go, line 134 at r5 (raw file):

	}

	// We need all the columns.

I think we do not need all the columns. Perhaps add a TODO to use a smaller set like the set of columns in the primary key. We basically want to create an insert event per row.


pkg/sql/distsqlrun/columnbackfiller.go, line 164 at r5 (raw file):

		}

		// TODO(vivek): is this necessary?

we don't need it!


pkg/sql/distsqlrun/processors.proto, line 360 at r4 (raw file):

  // backfiller to do its job, such as the descriptors for tables with fk
  // relationships to the table being modified.
  repeated sqlbase.TableDescriptor otherTables = 6 [(gogoproto.nullable) = false];

other_tables


pkg/sql/sqlbase/rowwriter.go, line 761 at r2 (raw file):

	Helper               rowHelper
	FetchCols            []ColumnDescriptor
	FetchColIDtoRowIndex map[ColumnID]int

Did you really need to export these as part of this PR?


Comments from Reviewable

@vivekmenezes
Copy link
Contributor

I took a first pass of this but would like to review runChunk() again.

I actually forget why we care about FKs for column backfill and will look into that tomorrow.


Review status: 15 of 20 files reviewed at latest revision, 10 unresolved discussions, some commit checks failed.


Comments from Reviewable

@jordanlewis jordanlewis force-pushed the dist-column-backfill-2 branch from f621ea4 to 19df378 Compare March 23, 2017 16:01
@jordanlewis
Copy link
Member Author

Review status: 13 of 20 files reviewed at latest revision, 10 unresolved discussions.


pkg/sql/backfill.go, line 144 at r5 (raw file):

Previously, vivekmenezes wrote…

we don't need addedColumnDescs anymore

Done.


pkg/sql/backfill.go, line 164 at r5 (raw file):

Previously, vivekmenezes wrote…

needColumnBackfill = true

we use the same backfill mechanism to delete a column

get rid of droppedColumnDescs

Done.


pkg/sql/distsqlrun/columnbackfiller.go, line 36 at r5 (raw file):

Previously, vivekmenezes wrote…

updateCols?

Added a comment.


pkg/sql/distsqlrun/columnbackfiller.go, line 41 at r5 (raw file):

Previously, vivekmenezes wrote…

and desc.Mutations that hold column descriptors.

I don't understand what you mean?


pkg/sql/distsqlrun/columnbackfiller.go, line 89 at r5 (raw file):

Previously, vivekmenezes wrote…

I think you meant to also append the mutation column to cols

Yes, thanks! This fixed the test issue :)


pkg/sql/distsqlrun/columnbackfiller.go, line 164 at r5 (raw file):

Previously, vivekmenezes wrote…

we don't need it!

Done.


pkg/sql/distsqlrun/processors.proto, line 360 at r4 (raw file):

Previously, vivekmenezes wrote…

other_tables

Done.


pkg/sql/sqlbase/rowwriter.go, line 761 at r2 (raw file):

Previously, vivekmenezes wrote…

Did you really need to export these as part of this PR?

Unfortunately, yes. All of these structs needed to move to sqlbase, and in order to do that, I needed make a bunch of these fields public.


Comments from Reviewable

@vivekmenezes
Copy link
Contributor

Review status: 13 of 20 files reviewed at latest revision, 10 unresolved discussions, some commit checks failed.


pkg/sql/distsqlrun/columnbackfiller.go, line 41 at r5 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

I don't understand what you mean?

the mutations can be either column or index mutations. But actually it's better the way it is because it is an index into Mutations[] independent of it hold. SO leave this as is.


Comments from Reviewable

// other_tables contains any other (leased) table descriptors necessary for the
// backfiller to do its job, such as the descriptors for tables with fk
// relationships to the table being modified.
repeated sqlbase.TableDescriptor other_tables = 6 [(gogoproto.nullable) = false];
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As discussed foreign key constraints can be checked before and after column backfill and so need not be passed to each processor.

@tamird
Copy link
Contributor

tamird commented Mar 23, 2017

3rd commit message lacks a verb: "add"?


Reviewed 5 of 5 files at r1, 13 of 13 files at r2, 1 of 1 files at r3, 7 of 7 files at r6, 5 of 5 files at r7.
Review status: all files reviewed at latest revision, 17 unresolved discussions, some commit checks failed.


pkg/sql/backfill.go, line 407 at r7 (raw file):

				return err
			}
			var fkTableDescs []sqlbase.TableDescriptor

this should be otherTableDescs, right? this might include other things in the future.


pkg/sql/distsql_physical_planner.go, line 604 at r7 (raw file):

	duration time.Duration,
	chunkSize int64,
	fkDescs []sqlbase.TableDescriptor,

otherTables?


pkg/sql/distsql_physical_planner.go, line 640 at r7 (raw file):

	chunkSize int64,
	spans []roachpb.Span,
	fkDescs []sqlbase.TableDescriptor,

otherTables?


pkg/sql/distsqlrun/columnbackfiller.go, line 1 at r7 (raw file):

// Copyright 2016 The Cockroach Authors.

2017


pkg/sql/distsqlrun/processors.proto, line 357 at r6 (raw file):

  optional int64 chunk_size = 5 [(gogoproto.nullable) = false];

  // other_tables contains any other (leased) table descriptors necessary for the

Looks like the local convention avoids naming the field in its comment.


pkg/sql/sqlbase/fk.go, line 32 at r2 (raw file):

type TableLookupsByID map[ID]TableLookup

// TableLookup is the value type of TableLookupsByID: An optional table

s/An/an/

but this comment would be better on each field rather than here.


Comments from Reviewable

@tamird
Copy link
Contributor

tamird commented Mar 23, 2017

Review status: all files reviewed at latest revision, 18 unresolved discussions, some commit checks failed.


pkg/sql/distsqlrun/columnbackfiller.go, line 64 at r7 (raw file):

			flowCtx: flowCtx,
			output:  output,
			spec:    *spec,

should this deref in the caller? that's where the nil check is.


Comments from Reviewable

@jordanlewis jordanlewis force-pushed the dist-column-backfill-2 branch from 19df378 to b0494c4 Compare March 24, 2017 00:02
@jordanlewis
Copy link
Member Author

Review status: 12 of 20 files reviewed at latest revision, 18 unresolved discussions.


pkg/sql/backfill.go, line 407 at r7 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

this should be otherTableDescs, right? this might include other things in the future.

Done.


pkg/sql/distsql_physical_planner.go, line 622 at r5 (raw file):

Previously, vivekmenezes wrote…

might as well create a generic distsqlrun.BackfillerSpec and only use the switch to set Type

Done.


pkg/sql/distsql_physical_planner.go, line 604 at r7 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

otherTables?

Done.


pkg/sql/distsql_physical_planner.go, line 640 at r7 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

otherTables?

Done.


pkg/sql/distsqlrun/columnbackfiller.go, line 1 at r7 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

2017

Done.


pkg/sql/distsqlrun/columnbackfiller.go, line 64 at r7 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

should this deref in the caller? that's where the nil check is.

Done.


pkg/sql/distsqlrun/processors.proto, line 357 at r6 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

Looks like the local convention avoids naming the field in its comment.

Done.


Comments from Reviewable

@tamird
Copy link
Contributor

tamird commented Mar 24, 2017

Reviewed 2 of 13 files at r2, 1 of 8 files at r8, 2 of 2 files at r9, 5 of 5 files at r10.
Review status: all files reviewed at latest revision, 12 unresolved discussions, some commit checks failed.


Comments from Reviewable

@RaduBerinde
Copy link
Member

Exciting stuff!


Review status: all files reviewed at latest revision, 12 unresolved discussions, some commit checks failed.


Comments from Reviewable

@vivekmenezes
Copy link
Contributor

vivekmenezes commented Mar 24, 2017 via email

@jordanlewis jordanlewis force-pushed the dist-column-backfill-2 branch 3 times, most recently from 55770a2 to c7df0b8 Compare March 29, 2017 19:33
@jordanlewis
Copy link
Member Author

PTAL!


Review status: 2 of 21 files reviewed at latest revision, 12 unresolved discussions, some commit checks pending.


pkg/sql/distsqlrun/columnbackfiller.go, line 134 at r5 (raw file):

Previously, vivekmenezes wrote…

I think we do not need all the columns. Perhaps add a TODO to use a smaller set like the set of columns in the primary key. We basically want to create an insert event per row.

There's already a todo for this in this file about tightening the bounds on the requestedCols down below in the rowUpdater. That todo encapsulates this request.


pkg/sql/distsqlrun/processors.proto, line 360 at r7 (raw file):

Previously, vivekmenezes wrote…

As discussed foreign key constraints can be checked before and after column backfill and so need not be passed to each processor.

I'll do this in a subsequent PR as discussed.


Comments from Reviewable

@vivekmenezes
Copy link
Contributor

:lgtm:


Review status: 2 of 21 files reviewed at latest revision, 17 unresolved discussions, all commit checks successful.


pkg/sql/backfill.go, line 144 at r16 (raw file):

				desc := m.GetColumn()
				if desc.DefaultExpr != nil || !desc.Nullable {
					needColumnBackfill = true

created this issue: #14455 to clean this up


pkg/sql/backfill.go, line 458 at r16 (raw file):

	version sqlbase.DescriptorVersion,
) error {
	return sc.distBackfill(ctx, lease, version, columnBackfill, columnTruncateAndBackfillChunkSize, distsqlrun.ColumnMutationFilter)

can you use getChunkSize(columnTruncateAndBackfillChunkSize)

please also fix the backfillIndexes case to use

getChunkSize(indexBackfillChunkSize) in a separate commit. Thanks!


pkg/sql/distsql_physical_planner.go, line 615 at r16 (raw file):

	case indexBackfill:
		ret.Type = distsqlrun.BackfillerSpec_Index
		return ret, nil

call return ret, nil at the bottom


pkg/sql/distsqlrun/columnbackfiller.go, line 82 at r16 (raw file):

	// If that's the case, and we end up reading a non-zero amount of data,
	// we need a throw an error since the old columns will already violate the
	// not null constraint.

TODO(jordan): fix #14455


pkg/sql/distsqlrun/columnbackfiller.go, line 132 at r16 (raw file):

	}

	// We need all the columns.

Added a TODO that we need to only read one of the columns forming the primary key. It's better not to make that change in this PR.


Comments from Reviewable

@vivekmenezes
Copy link
Contributor

#12424

@jordanlewis jordanlewis force-pushed the dist-column-backfill-2 branch from c7df0b8 to 2340266 Compare March 30, 2017 15:46
@jordanlewis
Copy link
Member Author

Review status: 2 of 21 files reviewed at latest revision, 17 unresolved discussions, all commit checks successful.


pkg/sql/backfill.go, line 458 at r16 (raw file):

Previously, vivekmenezes wrote…

can you use getChunkSize(columnTruncateAndBackfillChunkSize)

please also fix the backfillIndexes case to use

getChunkSize(indexBackfillChunkSize) in a separate commit. Thanks!

I'll fix these both in a follow on.


pkg/sql/distsql_physical_planner.go, line 615 at r16 (raw file):

Previously, vivekmenezes wrote…

call return ret, nil at the bottom

Done.


pkg/sql/distsqlrun/columnbackfiller.go, line 82 at r16 (raw file):

Previously, vivekmenezes wrote…

TODO(jordan): fix #14455

Done.


pkg/sql/distsqlrun/columnbackfiller.go, line 132 at r16 (raw file):

Previously, vivekmenezes wrote…

Added a TODO that we need to only read one of the columns forming the primary key. It's better not to make that change in this PR.

Hmm, I'm not sure that this is true. What if we're adding a new column that's part of the same column family as the rest of the columns? Granted the correct bounds for this is more subtle, but agreed that I'll do this in a separate PR.


Comments from Reviewable

makeDefaultExprs is currently used by insert, update, upsert and
backfill, which all live in the sql package. It will soon by used by
distsqlrun, so it needed to be moved somewhere importable by both sql
and distsqlrun - it now lives in sqlbase.
In preparation for being used by distsql.
Previously, the backfiller spec was passed to the index backfiller by
reference, despite it getting dereferenced immediately afterwards and
despite the caller being the one performing the nil check. Now,
dereference the spec right after the nil check is done.
This commit replaces the add/drop column backfill methods in
`backfill.go` with a new `chunkBackfiller` called `columnBackfiller`
that does the backfill via distsql.
@jordanlewis jordanlewis force-pushed the dist-column-backfill-2 branch from 2340266 to ad348ce Compare March 30, 2017 15:48
@vivekmenezes
Copy link
Contributor

Reviewed 2 of 13 files at r2, 2 of 13 files at r12, 19 of 21 files at r17.
Review status: 2 of 21 files reviewed at latest revision, 17 unresolved discussions, all commit checks successful.


pkg/sql/distsqlrun/columnbackfiller.go, line 132 at r16 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Hmm, I'm not sure that this is true. What if we're adding a new column that's part of the same column family as the rest of the columns? Granted the correct bounds for this is more subtle, but agreed that I'll do this in a separate PR.

ok


Comments from Reviewable

@jordanlewis
Copy link
Member Author

TFTRs!


Review status: 2 of 21 files reviewed at latest revision, 16 unresolved discussions, all commit checks successful.


Comments from Reviewable

@jordanlewis jordanlewis merged commit c2026d8 into cockroachdb:master Mar 30, 2017
@jordanlewis jordanlewis deleted the dist-column-backfill-2 branch March 30, 2017 18:19
@vivekmenezes
Copy link
Contributor

vivekmenezes commented Mar 30, 2017 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants