[BEAM-3773][SQL] Add EnumerableConverter for JDBC support#5173
[BEAM-3773][SQL] Add EnumerableConverter for JDBC support#5173kennknowles merged 1 commit intoapache:masterfrom
Conversation
|
R: @kennknowles |
kennknowles
left a comment
There was a problem hiding this comment.
Can this be tested in isolation at all? Or can it be tested via other test suites?
| } | ||
|
|
||
| public static Enumerable<Object> toEnumerable(BeamRelNode node) { | ||
| PipelineOptions options = PipelineOptionsFactory.create(); |
There was a problem hiding this comment.
Just checking - since Collector only works on the DirectRunner it seems fine to hardcode it here. But are the options specified elsewhere when using SQL so here it would just be validation that the configuration is supported?
There was a problem hiding this comment.
Pipelines that aren't supported should get a NullPointerException when the Collector tries to process an element. I considered adding a check but decided it could wait because this is the only public path in. Is the user able to change the runner without the ability to set the PipelineOptions?
There was a problem hiding this comment.
I'd rather get the options plumbed deliberately here. Assuming decent testing, it won't get left forever, but it does leave the code in a sort of weird stage. Does it add a ton of scope to plumb them?
There was a problem hiding this comment.
Plumbing the PipelineOptions into BeamEnumerableConverter is not easy as they will have to take a trip through the calcite JDBC interface on the way here (possibly on the jdbc connection string). I can add them as an argument to toEnumerable, but that will just move this line up to implement.
| } | ||
|
|
||
| private static class Collector extends DoFn<Row, Void> { | ||
| // This will only work on the direct runner. |
There was a problem hiding this comment.
It seems like there are two routes that are interesting here: (1) expose the ability to observe PCollection contents in the DirectRunner. This used to be the case when it only supported bounded collections. It is a different situation now, as the contents are never materialized. But that could change.
The other thing that would potentially make it cross runner, and what Scio does, is to write it to a sink. It could be TextIO writing to the globally-required tempLocation. We should definitely learn from Scio either way.
There was a problem hiding this comment.
I agree on writing to a sink. I was considering a version that sinks to BigQuery and read back from there. There are many options with different tradeoffs, we should consider them as followups.
There was a problem hiding this comment.
Yea, I think we probably want to be able to set options to control that if/when we do implement.
There was a problem hiding this comment.
Maybe just (w/ JIRA) add checkArgument(options.getRunner().getCanonicalName().equals("org.apache.beam.runners.direct.DirectRunner"))?
There was a problem hiding this comment.
Filed a jira: https://issues.apache.org/jira/browse/BEAM-4203
|
And this one is green. |
|
|
||
| Collector.globalValues.put(id, values); | ||
| run(options, node, new Collector()); | ||
| Collector.globalValues.remove(id); |
There was a problem hiding this comment.
Is it necessary to keep a static state? Cannot this be an instance field of the Collector?
There was a problem hiding this comment.
The Collector is serialized by beam and a copy is passed to the worker. As a result each worker ends up with a copy of the collector, so the original queue won't contain any values added by the worker. The static state is circumventing Beam's requirement that there be no global state. This only works in the direct runner because all workers are in the same process.
There was a problem hiding this comment.
It seems like a pattern of concern that this only works on the direct runner but maybe is positioned in the codebase in a way that makes us think it should work more broadly. It could be solved just by really clear signaling and validation, perhaps, and maybe TODO JIRAs? I'm looking for options to make everyone happy and unblock your other work.
There was a problem hiding this comment.
I'm explicitly violating the contract here, but being able to collect results in memory seems like a useful testing pattern for the direct runner. Is this something that might be able to live in a util folder inside the direct runner?
| /** | ||
| * BeamRelNode to replace a {@code Enumerable} node. | ||
| */ | ||
| public class BeamEnumerableConverter extends ConverterImpl implements EnumerableRel { |
There was a problem hiding this comment.
Can you document the behavior and how this class is used? And maybe an integration test of some kind?
There was a problem hiding this comment.
A integration test would be good, I'll get one added. I started down the unit test path last week and ended up deleting all the code that was unit testable when I found a library that implemented it.
| .apply(node.toPTransform()) | ||
| .apply(ParDo.of(doFn)); | ||
| PipelineResult result = pipeline.run(); | ||
| result.waitUntilFinish(); |
There was a problem hiding this comment.
Does this work in general case? E.g. what happens when the input is unbounded? My understanding is that Enumerables can be unbounded as well, and JDBC supports paginated unbounded results as well, is this going to work?
There was a problem hiding this comment.
This does not work in the general case, it only works on pipelines that eventually finish. I wanted to get something in for my DDL work, we can make this better in a followup.
| } | ||
|
|
||
| public static Enumerable<Object> toEnumerable(BeamRelNode node) { | ||
| PipelineOptions options = PipelineOptionsFactory.create(); |
There was a problem hiding this comment.
I'd rather get the options plumbed deliberately here. Assuming decent testing, it won't get left forever, but it does leave the code in a sort of weird stage. Does it add a ton of scope to plumb them?
| } | ||
|
|
||
| private static class Collector extends DoFn<Row, Void> { | ||
| // This will only work on the direct runner. |
There was a problem hiding this comment.
Yea, I think we probably want to be able to set options to control that if/when we do implement.
|
|
||
| Collector.globalValues.put(id, values); | ||
| run(options, node, new Collector()); | ||
| Collector.globalValues.remove(id); |
There was a problem hiding this comment.
It seems like a pattern of concern that this only works on the direct runner but maybe is positioned in the codebase in a way that makes us think it should work more broadly. It could be solved just by really clear signaling and validation, perhaps, and maybe TODO JIRAs? I'm looking for options to make everyone happy and unblock your other work.
kennknowles
left a comment
There was a problem hiding this comment.
Let's unblock w/ JIRAs as we all agree which piece to come back to.
| private static Enumerable<Object> count(PipelineOptions options, BeamRelNode node) { | ||
| PipelineResult result = run(options, node, new RowCounter()); | ||
| MetricQueryResults metrics = result.metrics().queryMetrics(MetricsFilter.builder() | ||
| .addNameFilter(MetricNameFilter.named(BeamEnumerableConverter.class, "rows")) |
There was a problem hiding this comment.
We probably want to spend some time on the best way to get reliable counts that are meaningful in the way that JDBC expects. I think the SQL <-> IO adapter may have to own it, for those Beam connectors that do things like write "mutations" that are not rows at all, etc.
There was a problem hiding this comment.
Created a jira: https://issues.apache.org/jira/browse/BEAM-4202
| } | ||
|
|
||
| private static class Collector extends DoFn<Row, Void> { | ||
| // This will only work on the direct runner. |
There was a problem hiding this comment.
Maybe just (w/ JIRA) add checkArgument(options.getRunner().getCanonicalName().equals("org.apache.beam.runners.direct.DirectRunner"))?
|
Fixed the issues from the comments and added tests at the |
This adds a converter from BeamRelNode to EnumerableRel. The calcite JDBC engine can execute any plan with a root node of the EnumerableRel type.
Follow this checklist to help us incorporate your contribution quickly and easily:
[BEAM-XXX] Fixes bug in ApproximateQuantiles, where you replaceBEAM-XXXwith the appropriate JIRA issue.mvn clean verifyto make sure basic checks pass. A more thorough check will be performed on your pull request automatically.