-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
distsql: elide RowChannel when connecting local processors #20550
Comments
Add BenchmarkRowChannelPipeline which benchmarks throughput and latency through a pipeline of RowChannels. The results are disappointing: name time/op RowChannelPipeline/1-8 110ns ± 2% RowChannelPipeline/2-8 272ns ± 1% RowChannelPipeline/3-8 377ns ± 4% RowChannelPipeline/4-8 421ns ± 1% name speed RowChannelPipeline/1-8 72.8MB/s ± 3% RowChannelPipeline/2-8 29.3MB/s ± 1% RowChannelPipeline/3-8 21.2MB/s ± 4% RowChannelPipeline/4-8 19.0MB/s ± 1% Release note: None See cockroachdb#20550 See cockroachdb#20553 See cockroachdb#20568
Add BenchmarkRowChannelPipeline which benchmarks throughput and latency through a pipeline of RowChannels. The results are disappointing: name time/op RowChannelPipeline/1-8 110ns ± 2% RowChannelPipeline/2-8 272ns ± 1% RowChannelPipeline/3-8 377ns ± 4% RowChannelPipeline/4-8 421ns ± 1% name speed RowChannelPipeline/1-8 72.8MB/s ± 3% RowChannelPipeline/2-8 29.3MB/s ± 1% RowChannelPipeline/3-8 21.2MB/s ± 4% RowChannelPipeline/4-8 19.0MB/s ± 1% Release note: None See cockroachdb#20550 See cockroachdb#20553 See cockroachdb#20568
@andreimatei brought up in #20584 the idea of "synchronous scheduling of co-located processors work". I'm not entirely sure what that means, but it seems to be an alternate design to this. I'd like to see it spelled out more explicitly. My thoughts: two reasons I like the design outlined in this issue to maintaining/improving the current design using different goroutines for different processors is that:
We definitely need benchmarks to convince anyone how direct function calls compare to using goroutines for separate processors. @danhhz has already done some preliminary work in this area, I'd also appreciate his thoughts here. [1]: http://db.csail.mit.edu/pubs/abadi-column-stores.pdf section 4.1 |
@arjunravinarayan I'd like to reiterate that fusing distsql processors together via JIT'ing is definitely not going to happen in 2018. There are lots of bits of lower hanging performance fruit to pluck first. And we also need to get distsql to feature parity with local sql. Definitely worthwhile to keep such fusing in mind, but I want to make sure any other spectators don't get too far ahead of the work immediately ahead. |
Refactor tableReader to implement the RowSource interface. Refactor tableReader.Run() to be implemented in terms of tableReader.Next() (i.e. the RowSource interface). See cockroachdb#20550 Release note: None
Refactor tableReader to implement the RowSource interface. Refactor tableReader.Run() to be implemented in terms of tableReader.Next() (i.e. the RowSource interface). Adjusted BenchmarkTableReader to avoid using a RowBuffer. This shows the benefit that can be achieved by using TableReader as a RowSource ("old" below is with the benchmark modified to use a RowChannel). name old time/op new time/op delta TableReader-8 11.6ms ± 5% 9.4ms ± 3% -18.81% (p=0.000 n=10+10) See cockroachdb#20550 Release note: None
After a day spent looking at profiles of distsql processors, it is clear that the lowest hanging fruit to pluck other than the overhead of Once processors implement |
Refactor tableReader to implement the RowSource interface. Refactor tableReader.Run() to be implemented in terms of tableReader.Next() (i.e. the RowSource interface). Adjusted BenchmarkTableReader to avoid using a RowBuffer. This shows the benefit that can be achieved by using TableReader as a RowSource ("old" below is with the benchmark modified to use a RowChannel). name old time/op new time/op delta TableReader-8 11.6ms ± 5% 9.4ms ± 3% -18.81% (p=0.000 n=10+10) See cockroachdb#20550 Release note: None
Refactor tableReader to implement the RowSource interface. Refactor tableReader.Run() to be implemented in terms of tableReader.Next() (i.e. the RowSource interface). Adjusted BenchmarkTableReader to avoid using a RowBuffer. This shows the benefit that can be achieved by using TableReader as a RowSource ("old" below is with the benchmark modified to use a RowChannel). name old time/op new time/op delta TableReader-8 11.6ms ± 5% 9.4ms ± 3% -18.81% (p=0.000 n=10+10) See cockroachdb#20550 Release note: None
Refactor tableReader to implement the RowSource interface. Refactor tableReader.Run() to be implemented in terms of tableReader.Next() (i.e. the RowSource interface). Adjusted BenchmarkTableReader to avoid using a RowBuffer. This shows the benefit that can be achieved by using TableReader as a RowSource ("old" below is with the benchmark modified to use a RowChannel). name old time/op new time/op delta TableReader-8 11.6ms ± 5% 9.4ms ± 3% -18.81% (p=0.000 n=10+10) See cockroachdb#20550 Release note: None
Refactor tableReader to implement the RowSource interface. Refactor tableReader.Run() to be implemented in terms of tableReader.Next() (i.e. the RowSource interface). Adjusted BenchmarkTableReader to avoid using a RowBuffer. This shows the benefit that can be achieved by using TableReader as a RowSource ("old" below is with the benchmark modified to use a RowChannel). name old time/op new time/op delta TableReader-8 11.6ms ± 5% 9.4ms ± 3% -18.81% (p=0.000 n=10+10) See cockroachdb#20550 Release note: None
Elide RowChannel when connecting local processors in simple cases. A simple `SELECT COUNT(*) FROM test.kv` query where `test.kv` contains 5m rows is reduced from 5.4s to 4.3s (a 20% speedup). When using distsql, this query utilizes a `tableReader` connected to an `aggregator`. For comparison, this query takes 4.5s when running with `set distsql=off`. These numbers are from a local single-node cluster. See cockroachdb#20550 Release note (performance improvement): Speed up distsql query execution by "fusing" processors executing on the same node together.
Elide RowChannel when connecting local processors in simple cases. A simple `SELECT COUNT(*) FROM test.kv` query where `test.kv` contains 5m rows is reduced from 5.4s to 4.3s (a 20% speedup). When using distsql, this query utilizes a `tableReader` connected to an `aggregator`. For comparison, this query takes 4.5s when running with `set distsql=off`. These numbers are from a local single-node cluster. See cockroachdb#20550 Release note (performance improvement): Speed up distsql query execution by "fusing" processors executing on the same node together.
Elide RowChannel when connecting local processors in simple cases. A simple `SELECT COUNT(*) FROM test.kv` query where `test.kv` contains 5m rows is reduced from 5.4s to 4.3s (a 20% speedup). When using distsql, this query utilizes a `tableReader` connected to an `aggregator`. For comparison, this query takes 4.5s when running with `set distsql=off`. These numbers are from a local single-node cluster. See cockroachdb#20550 Release note (performance improvement): Speed up distsql query execution by "fusing" processors executing on the same node together.
Elide RowChannel when connecting local processors in simple cases. A simple `SELECT COUNT(*) FROM test.kv` query where `test.kv` contains 5m rows is reduced from 5.4s to 4.3s (a 20% speedup). When using distsql, this query utilizes a `tableReader` connected to an `aggregator`. For comparison, this query takes 4.5s when running with `set distsql=off`. These numbers are from a local single-node cluster. See cockroachdb#20550 Release note (performance improvement): Speed up distsql query execution by "fusing" processors executing on the same node together.
Can this be closed, or are we waiting for a full solution? |
RowChannel
is currently used for connecting local distsqlProcessors
. ARowChannel
is essentially a channel with a fixed size. #19288 identifiedRowChannel
as a throughput bottleneck in distsql processing.RowChannel
implements both theRowSource
andRowReceiver
interfaces. EachProcessor
is run within a separate goroutine and theProcessor.Run
method loops, pulling rows from its inputs (RowSource
), processing them and emitting them viaProcOutputHelper
to aRowReceiver
.With a modest amount of refactoring, I think this could be restructured by having processor implementations also implement the
RowSource
interface. The internals of such a processor would be reorganized so thatNext()
would pull from its input(s), process and return the next row. This would be akin to theplanNode
interface. A processor which implementsRowSource
would either be run via a call toRun()
, in which caseNext()
would never be called, or zero or more calls toNext()
. That is, a processor would either act as aRowSource
or as aProcessor
within a given flow, never both.We can incrementally move towards this new API. Currently,
Flow.setup()
always joins processors together with aRowChannel
orMultiplexedRowChannel
. If a processor implementsRowSource
, creation of theRowChannel
can be elided. We would have to mark processors joined in this way and only callProcessor.Run
on processors that are not acting asRowSources
.See also #19134 which mentions "fusing" multiple processors into a single goroutine. The above proposal would achieve this.
This proposal needs to be validated by benchmarking before significant work is done. There are undoubtedly complexities with regards to cancellation and the particulars of the
RowSource
interface that will make this challenging.@arjunravinarayan, @asubiotto You've likely already been thinking in this direction, but I couldn't find an issue discussing this.
The text was updated successfully, but these errors were encountered: