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

importccl: implement EXPORT csv #24755

Closed
wants to merge 2 commits into
base: master
from

Conversation

Projects
None yet
4 participants
@dt
Member

dt commented Apr 12, 2018

This adds a DistSQL processor that writes whatever it is given as CSV files to a specified storage location.

This processor can be appended as a sink for arbitrary SELECT queries.

The primary intended use-case is dumping tables as CSV for bulk-export, but it can also be used on any SELECT query, so it could also be used in report generation or other workflows.

Release note (enterprise change): prototype support for EXPORT CSV.

@dt dt requested review from mjibson, danhhz and arjunravinarayan Apr 12, 2018

@dt dt requested review from cockroachdb/distsql-prs as code owners Apr 12, 2018

@cockroach-teamcity

This comment has been minimized.

Show comment
Hide comment
@cockroach-teamcity

cockroach-teamcity Apr 12, 2018

Member

This change is Reviewable

Member

cockroach-teamcity commented Apr 12, 2018

This change is Reviewable

@dt dt changed the title from [WIP] importccl: implement EXPORT csv to importccl: implement EXPORT csv Apr 16, 2018

Show outdated Hide outdated pkg/ccl/importccl/exportcsv.go Outdated
Show outdated Hide outdated pkg/ccl/importccl/exportcsv.go Outdated
Show outdated Hide outdated pkg/ccl/importccl/exportcsv.go Outdated
Show outdated Hide outdated pkg/ccl/importccl/exportcsv.go Outdated
Show outdated Hide outdated pkg/ccl/importccl/exportcsv.go Outdated
Show outdated Hide outdated pkg/sql/distsqlrun/processors.proto Outdated

@dt dt requested a review from andreimatei Apr 18, 2018

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 19, 2018

Member

I think this is just about RFAL.

In particular, I'd appreciate DistSQL folks (@andreimatei @arjunravinarayan) taking a look at PlanAndRunExport and my csvWriter processor. My hope is that this processor is fused to its inputs and my understanding is that I don't need to do anything special to allow that?

Member

dt commented Apr 19, 2018

I think this is just about RFAL.

In particular, I'd appreciate DistSQL folks (@andreimatei @arjunravinarayan) taking a look at PlanAndRunExport and my csvWriter processor. My hope is that this processor is fused to its inputs and my understanding is that I don't need to do anything special to allow that?

@mjibson

This comment has been minimized.

Show comment
Hide comment
@mjibson

mjibson Apr 19, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 127 at r7 (raw file):

		chunk := exportChunkSizeDefault
		if override, ok := opts[exportOptionChunkSize]; ok {
			chunk, err = strconv.Atoi(override)

I suppose we should at least check that chunk >= 0.


pkg/ccl/importccl/exportcsv.go, line 214 at r7 (raw file):

		var buf bytes.Buffer
		writer := csv.NewWriter(&buf)
		if sp.spec.Delimiter != 0 {

sp.spec.Delimiter should never be 0, so we should remove this if line. If it is somehow 0, then the go csv writer will fail because it has a check for that. And with the current code if it is somehow 0, then this will actually set it to comma, which isn't correct either.


pkg/ccl/importccl/exportcsv.go, line 218 at r7 (raw file):

		}

		f := tree.NewFmtCtxWithBuf(tree.FmtBareStrings)

Add a TODO here for me to fix this once my change is in since this won't work for string datums with non-ascii.


pkg/ccl/importccl/exportcsv.go, line 243 at r7 (raw file):

				for i, ed := range row {
					if err := ed.EnsureDecoded(&types[i], alloc); err != nil {

Does it make sense to reassign alloc to an empty datumalloc after a csv write so the memory can be GC'd? Otherwise I think we'll hold all datums in memory for the entire table reader.


pkg/ccl/importccl/exportcsv_test.go, line 9 at r7 (raw file):

//     https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package importccl_test

Could you add another test that doesn't use tables at all just to make sure? Like a select generate_series(1, 5) or something.


Comments from Reviewable

Member

mjibson commented Apr 19, 2018

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


pkg/ccl/importccl/exportcsv.go, line 127 at r7 (raw file):

		chunk := exportChunkSizeDefault
		if override, ok := opts[exportOptionChunkSize]; ok {
			chunk, err = strconv.Atoi(override)

I suppose we should at least check that chunk >= 0.


pkg/ccl/importccl/exportcsv.go, line 214 at r7 (raw file):

		var buf bytes.Buffer
		writer := csv.NewWriter(&buf)
		if sp.spec.Delimiter != 0 {

sp.spec.Delimiter should never be 0, so we should remove this if line. If it is somehow 0, then the go csv writer will fail because it has a check for that. And with the current code if it is somehow 0, then this will actually set it to comma, which isn't correct either.


pkg/ccl/importccl/exportcsv.go, line 218 at r7 (raw file):

		}

		f := tree.NewFmtCtxWithBuf(tree.FmtBareStrings)

Add a TODO here for me to fix this once my change is in since this won't work for string datums with non-ascii.


pkg/ccl/importccl/exportcsv.go, line 243 at r7 (raw file):

				for i, ed := range row {
					if err := ed.EnsureDecoded(&types[i], alloc); err != nil {

Does it make sense to reassign alloc to an empty datumalloc after a csv write so the memory can be GC'd? Otherwise I think we'll hold all datums in memory for the entire table reader.


pkg/ccl/importccl/exportcsv_test.go, line 9 at r7 (raw file):

//     https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package importccl_test

Could you add another test that doesn't use tables at all just to make sure? Like a select generate_series(1, 5) or something.


Comments from Reviewable

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 19, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 127 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

I suppose we should at least check that chunk >= 0.

Done.


pkg/ccl/importccl/exportcsv.go, line 214 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

sp.spec.Delimiter should never be 0, so we should remove this if line. If it is somehow 0, then the go csv writer will fail because it has a check for that. And with the current code if it is somehow 0, then this will actually set it to comma, which isn't correct either.

Done.


pkg/ccl/importccl/exportcsv.go, line 218 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

Add a TODO here for me to fix this once my change is in since this won't work for string datums with non-ascii.

Done.


pkg/ccl/importccl/exportcsv.go, line 243 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

Does it make sense to reassign alloc to an empty datumalloc after a csv write so the memory can be GC'd? Otherwise I think we'll hold all datums in memory for the entire table reader.

I thought it internally moved it's slice header along to avoid that?


pkg/ccl/importccl/exportcsv_test.go, line 9 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

Could you add another test that doesn't use tables at all just to make sure? Like a select generate_series(1, 5) or something.

that's unsupported by distsql, and thus by EXPORT... but its good to have a negative test.
Done.


Comments from Reviewable

Member

dt commented Apr 19, 2018

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


pkg/ccl/importccl/exportcsv.go, line 127 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

I suppose we should at least check that chunk >= 0.

Done.


pkg/ccl/importccl/exportcsv.go, line 214 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

sp.spec.Delimiter should never be 0, so we should remove this if line. If it is somehow 0, then the go csv writer will fail because it has a check for that. And with the current code if it is somehow 0, then this will actually set it to comma, which isn't correct either.

Done.


pkg/ccl/importccl/exportcsv.go, line 218 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

Add a TODO here for me to fix this once my change is in since this won't work for string datums with non-ascii.

Done.


pkg/ccl/importccl/exportcsv.go, line 243 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

Does it make sense to reassign alloc to an empty datumalloc after a csv write so the memory can be GC'd? Otherwise I think we'll hold all datums in memory for the entire table reader.

I thought it internally moved it's slice header along to avoid that?


pkg/ccl/importccl/exportcsv_test.go, line 9 at r7 (raw file):

Previously, mjibson (Matt Jibson) wrote…

Could you add another test that doesn't use tables at all just to make sure? Like a select generate_series(1, 5) or something.

that's unsupported by distsql, and thus by EXPORT... but its good to have a negative test.
Done.


Comments from Reviewable

@mjibson

This comment has been minimized.

Show comment
Hide comment
@mjibson

mjibson Apr 19, 2018

Member

:lgtm:


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


pkg/ccl/importccl/exportcsv.go, line 243 at r7 (raw file):

Previously, dt (David Taylor) wrote…

I thought it internally moved it's slice header along to avoid that?

Oh it does! I read the code too quickly before and didn't realize it was that smart. Cool.


Comments from Reviewable

Member

mjibson commented Apr 19, 2018

:lgtm:


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


pkg/ccl/importccl/exportcsv.go, line 243 at r7 (raw file):

Previously, dt (David Taylor) wrote…

I thought it internally moved it's slice header along to avoid that?

Oh it does! I read the code too quickly before and didn't realize it was that smart. Cool.


Comments from Reviewable

@andreimatei

This comment has been minimized.

Show comment
Hide comment
@andreimatei

andreimatei Apr 19, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

func (sp *csvWriter) Run(ctx context.Context, wg *sync.WaitGroup) {
	ctx, span := tracing.ChildSpan(ctx, "csvWriter")

you probably want to embed processorBase and use startInternal to get a potentially recording span.


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

			}
			if cs != distsqlrun.NeedMoreRows {
				return errors.New("unexpected closure of consumer")

why unexpected? If your input produced an error (which bypassed you cause you use a NoMetadataRowSource, then the consumer is gonna start draining.
You might want to use emitHelper().

Or these days generally the recommendation is to have your processor implement RowSource, in which case you implement a Next() instead of Run() and have other utilities to help you. I didn't tell you to do it yesterday cause you had already written the thing and I was at the time merging some of these utilities, but because of the tracing issue and this consumer status issue here, I think you might want to do that after all. See comments on ProcessorBase for patterns.
Up to you though.


pkg/ccl/importccl/exportcsv.go, line 315 at r8 (raw file):

	distsqlrun.DrainAndClose(
		ctx, sp.output, err, func(context.Context) {} /* pushTrailingMeta */, sp.input)

no tracing info in trailing meta?


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

	)

	// this mirrors the call in addAggregators for non-multi-stage.

I don't know what this comment is telling me, even after trying to follow the pointer.
Do you need a planToStreamColMap? Is there a corresponding plan node to this processor? Doesn't look like it.


pkg/sql/distsqlrun/processors.proto, line 561 at r8 (raw file):

  optional string name_pattern = 2 [(gogoproto.nullable) = false];
  optional int32 delimiter = 3 [(gogoproto.nullable) = false];
  // chunk_rows = 0 indicated no limit.

and if it's not zero, what does it mean? And what's a "span" in the above description?


Comments from Reviewable

Member

andreimatei commented Apr 19, 2018

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

func (sp *csvWriter) Run(ctx context.Context, wg *sync.WaitGroup) {
	ctx, span := tracing.ChildSpan(ctx, "csvWriter")

you probably want to embed processorBase and use startInternal to get a potentially recording span.


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

			}
			if cs != distsqlrun.NeedMoreRows {
				return errors.New("unexpected closure of consumer")

why unexpected? If your input produced an error (which bypassed you cause you use a NoMetadataRowSource, then the consumer is gonna start draining.
You might want to use emitHelper().

Or these days generally the recommendation is to have your processor implement RowSource, in which case you implement a Next() instead of Run() and have other utilities to help you. I didn't tell you to do it yesterday cause you had already written the thing and I was at the time merging some of these utilities, but because of the tracing issue and this consumer status issue here, I think you might want to do that after all. See comments on ProcessorBase for patterns.
Up to you though.


pkg/ccl/importccl/exportcsv.go, line 315 at r8 (raw file):

	distsqlrun.DrainAndClose(
		ctx, sp.output, err, func(context.Context) {} /* pushTrailingMeta */, sp.input)

no tracing info in trailing meta?


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

	)

	// this mirrors the call in addAggregators for non-multi-stage.

I don't know what this comment is telling me, even after trying to follow the pointer.
Do you need a planToStreamColMap? Is there a corresponding plan node to this processor? Doesn't look like it.


pkg/sql/distsqlrun/processors.proto, line 561 at r8 (raw file):

  optional string name_pattern = 2 [(gogoproto.nullable) = false];
  optional int32 delimiter = 3 [(gogoproto.nullable) = false];
  // chunk_rows = 0 indicated no limit.

and if it's not zero, what does it mean? And what's a "span" in the above description?


Comments from Reviewable

sql: remove DistLoader wrapper
I concur with Dan’s year-old TODO: DistLoader doesn’t seem like
a useful abstraction and we’re better off just getting an unwrapped
DistSQLPlanner into the caller’s hands so they can use its public API
without fighting though additional indirection.

Indeed, Ideally I’d like to follow this with exporting more of the
DistSQL API, with the goal of eventually moving LoadCSV method to a
function in importccl defined entirely in terms of the public DistSQL
API, though that will obviously require some refactoring and careful
thinking about where to draw maintainable lines between these packages.

Release note: none.
@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 24, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

you probably want to embed processorBase and use startInternal to get a potentially recording span.

We discussed this in-person last week and there are lots of things we'd need to export to get to that point -- I thought we'd concluded we'd go ahead and make this processor match the other *ccl distsql processors, then do a single refactor that flips them all over along with the API changes to distsql's packages?


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

why unexpected? If your input produced an error (which bypassed you cause you use a NoMetadataRowSource, then the consumer is gonna start draining.
You might want to use emitHelper().

Or these days generally the recommendation is to have your processor implement RowSource, in which case you implement a Next() instead of Run() and have other utilities to help you. I didn't tell you to do it yesterday cause you had already written the thing and I was at the time merging some of these utilities, but because of the tracing issue and this consumer status issue here, I think you might want to do that after all. See comments on ProcessorBase for patterns.
Up to you though.

Hmm, this mirrors what we do in the other *ccl processors. AFAIK, the only way the EXPORT plan wouldn't want more rows is if something has already gone wrong, so I think that isn't expected, so it is fine to error here rather than continuing?

re:RowSource as mentioned above, I thought we were skipping RowSource for now, and then looking at converting all the *ccl ones at once.


pkg/ccl/importccl/exportcsv.go, line 315 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

no tracing info in trailing meta?

mirrors the other *ccl distsql plans. I'll add a TODO.


Comments from Reviewable

Member

dt commented Apr 24, 2018

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

you probably want to embed processorBase and use startInternal to get a potentially recording span.

We discussed this in-person last week and there are lots of things we'd need to export to get to that point -- I thought we'd concluded we'd go ahead and make this processor match the other *ccl distsql processors, then do a single refactor that flips them all over along with the API changes to distsql's packages?


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

why unexpected? If your input produced an error (which bypassed you cause you use a NoMetadataRowSource, then the consumer is gonna start draining.
You might want to use emitHelper().

Or these days generally the recommendation is to have your processor implement RowSource, in which case you implement a Next() instead of Run() and have other utilities to help you. I didn't tell you to do it yesterday cause you had already written the thing and I was at the time merging some of these utilities, but because of the tracing issue and this consumer status issue here, I think you might want to do that after all. See comments on ProcessorBase for patterns.
Up to you though.

Hmm, this mirrors what we do in the other *ccl processors. AFAIK, the only way the EXPORT plan wouldn't want more rows is if something has already gone wrong, so I think that isn't expected, so it is fine to error here rather than continuing?

re:RowSource as mentioned above, I thought we were skipping RowSource for now, and then looking at converting all the *ccl ones at once.


pkg/ccl/importccl/exportcsv.go, line 315 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

no tracing info in trailing meta?

mirrors the other *ccl distsql plans. I'll add a TODO.


Comments from Reviewable

importccl: implement EXPORT CSV
This adds a DistSQL processor that writes whatever it is given as CSV
files to a specified storage location.

This processor can be appended as a sink for arbitrary SELECT queries.

The primary intended use-case is dumping tables as CSV for bulk-export,
but it can also be used on any SELECT query, so it could also be used in
report generation or other workflows.

Release note (enterprise change): prototype support for EXPORT CSV.
@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 24, 2018

Member

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


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I don't know what this comment is telling me, even after trying to follow the pointer.
Do you need a planToStreamColMap? Is there a corresponding plan node to this processor? Doesn't look like it.

Updated comment.


pkg/sql/distsqlrun/processors.proto, line 561 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

and if it's not zero, what does it mean? And what's a "span" in the above description?

Done.


Comments from Reviewable

Member

dt commented Apr 24, 2018

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


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I don't know what this comment is telling me, even after trying to follow the pointer.
Do you need a planToStreamColMap? Is there a corresponding plan node to this processor? Doesn't look like it.

Updated comment.


pkg/sql/distsqlrun/processors.proto, line 561 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

and if it's not zero, what does it mean? And what's a "span" in the above description?

Done.


Comments from Reviewable

@andreimatei

This comment has been minimized.

Show comment
Hide comment
@andreimatei

andreimatei Apr 24, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

Previously, dt (David Taylor) wrote…

We discussed this in-person last week and there are lots of things we'd need to export to get to that point -- I thought we'd concluded we'd go ahead and make this processor match the other *ccl distsql processors, then do a single refactor that flips them all over along with the API changes to distsql's packages?

well I wasn't thinking about tracing at that point. This is not the most important thing in the world and you don't technically need to use any utilities that you don't want to (you can do the tracing work yourself).


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, dt (David Taylor) wrote…

Hmm, this mirrors what we do in the other *ccl processors. AFAIK, the only way the EXPORT plan wouldn't want more rows is if something has already gone wrong, so I think that isn't expected, so it is fine to error here rather than continuing?

re:RowSource as mentioned above, I thought we were skipping RowSource for now, and then looking at converting all the *ccl ones at once.

It may be the case that the consumer would only ask you to drain if something has gone wrong somewhere, but that's no reason send an(other) error. So if we're so confident that the this proc's consumer has already received an error, it'd be better to return nil, nil rather than this. But what you really want, I think, is to send some trace info. And also you want to drain your input - e.g. the TableReader will give you some TxnMeta and misplanned ranges info which is good to pass along.
Basically, all these procs should follow a pattern, as much as possible. Regardless of what interfaces you implement and what utilities you use, respecting contracts such as draining your input I think is important.


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, dt (David Taylor) wrote…

Updated comment.

I'm not sure I follow - if there' s no planNode corresponding to this proc (or rather nobody's consuming this p) , then do we need to care about the planToStreamColMap?


Comments from Reviewable

Member

andreimatei commented Apr 24, 2018

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

Previously, dt (David Taylor) wrote…

We discussed this in-person last week and there are lots of things we'd need to export to get to that point -- I thought we'd concluded we'd go ahead and make this processor match the other *ccl distsql processors, then do a single refactor that flips them all over along with the API changes to distsql's packages?

well I wasn't thinking about tracing at that point. This is not the most important thing in the world and you don't technically need to use any utilities that you don't want to (you can do the tracing work yourself).


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, dt (David Taylor) wrote…

Hmm, this mirrors what we do in the other *ccl processors. AFAIK, the only way the EXPORT plan wouldn't want more rows is if something has already gone wrong, so I think that isn't expected, so it is fine to error here rather than continuing?

re:RowSource as mentioned above, I thought we were skipping RowSource for now, and then looking at converting all the *ccl ones at once.

It may be the case that the consumer would only ask you to drain if something has gone wrong somewhere, but that's no reason send an(other) error. So if we're so confident that the this proc's consumer has already received an error, it'd be better to return nil, nil rather than this. But what you really want, I think, is to send some trace info. And also you want to drain your input - e.g. the TableReader will give you some TxnMeta and misplanned ranges info which is good to pass along.
Basically, all these procs should follow a pattern, as much as possible. Regardless of what interfaces you implement and what utilities you use, respecting contracts such as draining your input I think is important.


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, dt (David Taylor) wrote…

Updated comment.

I'm not sure I follow - if there' s no planNode corresponding to this proc (or rather nobody's consuming this p) , then do we need to care about the planToStreamColMap?


Comments from Reviewable

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 24, 2018

Member

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


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I'm not sure I follow - if there' s no planNode corresponding to this proc (or rather nobody's consuming this p) , then do we need to care about the planToStreamColMap?

There is a plan -- the one that called this method. It expects the passed RowResultWriter to get the rows produced by processor spec'ed by out.


Comments from Reviewable

Member

dt commented Apr 24, 2018

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


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

I'm not sure I follow - if there' s no planNode corresponding to this proc (or rather nobody's consuming this p) , then do we need to care about the planToStreamColMap?

There is a plan -- the one that called this method. It expects the passed RowResultWriter to get the rows produced by processor spec'ed by out.


Comments from Reviewable

@andreimatei

This comment has been minimized.

Show comment
Hide comment
@andreimatei

andreimatei Apr 24, 2018

Member

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


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, dt (David Taylor) wrote…

There is a plan -- the one that called this method. It expects the passed RowResultWriter to get the rows produced by processor spec'ed by out.

Oh I see that distSQLReceiver uses this map. That makes sense.
The comment still isn't telling me anything though (also, grammar error - with to?) so I suggest:

// Overwrite planToStreamColMap (used by recv below) to reflect the output of the non-grouping stage 
// we've added above. That stage outputs produces columns filename/rows/bytes.

Comments from Reviewable

Member

andreimatei commented Apr 24, 2018

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


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, dt (David Taylor) wrote…

There is a plan -- the one that called this method. It expects the passed RowResultWriter to get the rows produced by processor spec'ed by out.

Oh I see that distSQLReceiver uses this map. That makes sense.
The comment still isn't telling me anything though (also, grammar error - with to?) so I suggest:

// Overwrite planToStreamColMap (used by recv below) to reflect the output of the non-grouping stage 
// we've added above. That stage outputs produces columns filename/rows/bytes.

Comments from Reviewable

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 25, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

well I wasn't thinking about tracing at that point. This is not the most important thing in the world and you don't technically need to use any utilities that you don't want to (you can do the tracing work yourself).

i think my point below applies here too.


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

It may be the case that the consumer would only ask you to drain if something has gone wrong somewhere, but that's no reason send an(other) error. So if we're so confident that the this proc's consumer has already received an error, it'd be better to return nil, nil rather than this. But what you really want, I think, is to send some trace info. And also you want to drain your input - e.g. the TableReader will give you some TxnMeta and misplanned ranges info which is good to pass along.
Basically, all these procs should follow a pattern, as much as possible. Regardless of what interfaces you implement and what utilities you use, respecting contracts such as draining your input I think is important.

I agree with following a pattern and being consistent, and right now, this one is following the pattern of the other *ccl processors.

It sounds like there have been some changes in the best practices for procs that didn't quite get applied to the existing *ccl procs. It also sounds like they're usually implemented with some common helpers, and maybe we should update the existing *ccl procs to use those helpers and abide by the current best practices, which might require also some refactoring in distsql's public API so those helpers can be reused.

In the short term though, I think I'd rather follow the pattern of the other *ccl procs, and let future metadata/tracing/etc fixes be applied to this one along with them, rather than adding the distsql refactors to this change now?


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

Oh I see that distSQLReceiver uses this map. That makes sense.
The comment still isn't telling me anything though (also, grammar error - with to?) so I suggest:

// Overwrite planToStreamColMap (used by recv below) to reflect the output of the non-grouping stage 
// we've added above. That stage outputs produces columns filename/rows/bytes.

Done.


Comments from Reviewable

Member

dt commented Apr 25, 2018

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


pkg/ccl/importccl/exportcsv.go, line 197 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

well I wasn't thinking about tracing at that point. This is not the most important thing in the world and you don't technically need to use any utilities that you don't want to (you can do the tracing work yourself).

i think my point below applies here too.


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

It may be the case that the consumer would only ask you to drain if something has gone wrong somewhere, but that's no reason send an(other) error. So if we're so confident that the this proc's consumer has already received an error, it'd be better to return nil, nil rather than this. But what you really want, I think, is to send some trace info. And also you want to drain your input - e.g. the TableReader will give you some TxnMeta and misplanned ranges info which is good to pass along.
Basically, all these procs should follow a pattern, as much as possible. Regardless of what interfaces you implement and what utilities you use, respecting contracts such as draining your input I think is important.

I agree with following a pattern and being consistent, and right now, this one is following the pattern of the other *ccl processors.

It sounds like there have been some changes in the best practices for procs that didn't quite get applied to the existing *ccl procs. It also sounds like they're usually implemented with some common helpers, and maybe we should update the existing *ccl procs to use those helpers and abide by the current best practices, which might require also some refactoring in distsql's public API so those helpers can be reused.

In the short term though, I think I'd rather follow the pattern of the other *ccl procs, and let future metadata/tracing/etc fixes be applied to this one along with them, rather than adding the distsql refactors to this change now?


pkg/sql/distsql_plan_csv.go, line 66 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

Oh I see that distSQLReceiver uses this map. That makes sense.
The comment still isn't telling me anything though (also, grammar error - with to?) so I suggest:

// Overwrite planToStreamColMap (used by recv below) to reflect the output of the non-grouping stage 
// we've added above. That stage outputs produces columns filename/rows/bytes.

Done.


Comments from Reviewable

@dt dt closed this Apr 25, 2018

@dt dt deleted the dt:export branch Apr 25, 2018

@andreimatei

This comment has been minimized.

Show comment
Hide comment
@andreimatei

andreimatei Apr 25, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, dt (David Taylor) wrote…

I agree with following a pattern and being consistent, and right now, this one is following the pattern of the other *ccl processors.

It sounds like there have been some changes in the best practices for procs that didn't quite get applied to the existing *ccl procs. It also sounds like they're usually implemented with some common helpers, and maybe we should update the existing *ccl procs to use those helpers and abide by the current best practices, which might require also some refactoring in distsql's public API so those helpers can be reused.

In the short term though, I think I'd rather follow the pattern of the other *ccl procs, and let future metadata/tracing/etc fixes be applied to this one along with them, rather than adding the distsql refactors to this change now?

There's been changes to the procs that implement RowSource (and there's generally a desire for everyone to implement it).
Processors that don't have always had a different set of helpers - emitHelper(), DrainAndClose(). Regardless of what helper you do or don't use, you have to drain your input. You have been warned :)


Comments from Reviewable

Member

andreimatei commented Apr 25, 2018

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


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, dt (David Taylor) wrote…

I agree with following a pattern and being consistent, and right now, this one is following the pattern of the other *ccl processors.

It sounds like there have been some changes in the best practices for procs that didn't quite get applied to the existing *ccl procs. It also sounds like they're usually implemented with some common helpers, and maybe we should update the existing *ccl procs to use those helpers and abide by the current best practices, which might require also some refactoring in distsql's public API so those helpers can be reused.

In the short term though, I think I'd rather follow the pattern of the other *ccl procs, and let future metadata/tracing/etc fixes be applied to this one along with them, rather than adding the distsql refactors to this change now?

There's been changes to the procs that implement RowSource (and there's generally a desire for everyone to implement it).
Processors that don't have always had a different set of helpers - emitHelper(), DrainAndClose(). Regardless of what helper you do or don't use, you have to drain your input. You have been warned :)


Comments from Reviewable

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 25, 2018

Member

closed by mistaken push -d instead of push -f and apparently that precludes re-opening because the branch changed while it was closed (obviously). ugh.

Member

dt commented Apr 25, 2018

closed by mistaken push -d instead of push -f and apparently that precludes re-opening because the branch changed while it was closed (obviously). ugh.

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 25, 2018

Member

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


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

There's been changes to the procs that implement RowSource (and there's generally a desire for everyone to implement it).
Processors that don't have always had a different set of helpers - emitHelper(), DrainAndClose(). Regardless of what helper you do or don't use, you have to drain your input. You have been warned :)

I think we do DrainAndClose right below this -- this whole block is in a err := func() error {...}() and that err is passed to DrainAndClose, so I think it is called unconditionally, right?


Comments from Reviewable

Member

dt commented Apr 25, 2018

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


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

There's been changes to the procs that implement RowSource (and there's generally a desire for everyone to implement it).
Processors that don't have always had a different set of helpers - emitHelper(), DrainAndClose(). Regardless of what helper you do or don't use, you have to drain your input. You have been warned :)

I think we do DrainAndClose right below this -- this whole block is in a err := func() error {...}() and that err is passed to DrainAndClose, so I think it is called unconditionally, right?


Comments from Reviewable

@andreimatei

This comment has been minimized.

Show comment
Hide comment
@andreimatei

andreimatei Apr 25, 2018

Member

:lgtm:


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


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, dt (David Taylor) wrote…

I think we do DrainAndClose right below this -- this whole block is in a err := func() error {...}() and that err is passed to DrainAndClose, so I think it is called unconditionally, right?

ah, right, I didn't notice we're in an anonymous function. Consider making it a method.


Comments from Reviewable

Member

andreimatei commented Apr 25, 2018

:lgtm:


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


pkg/ccl/importccl/exportcsv.go, line 304 at r8 (raw file):

Previously, dt (David Taylor) wrote…

I think we do DrainAndClose right below this -- this whole block is in a err := func() error {...}() and that err is passed to DrainAndClose, so I think it is called unconditionally, right?

ah, right, I didn't notice we're in an anonymous function. Consider making it a method.


Comments from Reviewable

@dt dt referenced this pull request Apr 25, 2018

Merged

importccl: EXPORT CSV #25075

@dt

This comment has been minimized.

Show comment
Hide comment
@dt

dt Apr 25, 2018

Member

re-opened as PR #25075

Member

dt commented Apr 25, 2018

re-opened as PR #25075

craig bot pushed a commit that referenced this pull request Apr 26, 2018

Merge #25075 #25107
25075: importccl: EXPORT CSV r=dt a=dt

This is a re-opening of #24755 after it was closed by a mistaken branch deletion, which unfortunately also prevents re-opening in place.

This adds a DistSQL processor that writes whatever it is given as CSV files to a specified storage location.

This processor can be appended as a sink for arbitrary SELECT queries.

The primary intended use-case is dumping tables as CSV for bulk-export, but it can also be used on any SELECT query, so it could also be used in report generation or other workflows.

Release note (enterprise change): prototype support for EXPORT CSV.


25107: sql: cleanup reference to ExecCfg r=andreimatei a=andreimatei

A test was using an InternalExecutor to get an ExecutorConfig, but
there's a better way. Also the old way is about to break.

Release note: None

Co-authored-by: David Taylor <tinystatemachine@gmail.com>
Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>

craig bot pushed a commit that referenced this pull request Apr 26, 2018

Merge #25075
25075: importccl: EXPORT CSV r=dt a=dt

This is a re-opening of #24755 after it was closed by a mistaken branch deletion, which unfortunately also prevents re-opening in place.

This adds a DistSQL processor that writes whatever it is given as CSV files to a specified storage location.

This processor can be appended as a sink for arbitrary SELECT queries.

The primary intended use-case is dumping tables as CSV for bulk-export, but it can also be used on any SELECT query, so it could also be used in report generation or other workflows.

Release note (enterprise change): prototype support for EXPORT CSV.


Co-authored-by: David Taylor <tinystatemachine@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment