[BEAM-2436] table is not regeisted in BeamSql.query#3342
[BEAM-2436] table is not regeisted in BeamSql.query#3342mingmxu wants to merge 1 commit intoapache:DSL_SQLfrom
Conversation
| PCollection<BeamSQLRow> outputStream = inputTable.apply(BeamSql.simpleQuery(sql)); | ||
| //Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery; | ||
| PCollection<BeamSQLRow> outputStream = inputTable.apply( | ||
| BeamSql.simpleQuery("select c2, c3 from TABLE_A where c1=1")); |
There was a problem hiding this comment.
The usage of BeamSql.simpleQuery seems a little weird: user have never specified the table name: TABLE_A.
There was a problem hiding this comment.
simpleQuery is added for single table SQL. As it's not possible to name a PCollection here, it's automately set to table name used in query.
This brings a potential issue of table name conflict, the solution may be limiting the scope of table schema, will open a new task to talk.
There was a problem hiding this comment.
Ok, let's talk further in the new task.
There was a problem hiding this comment.
Its extremely convenient to not have to name your tables in the simpleQuery approach.
There was a problem hiding this comment.
@takidau as talking about the design of interface, the default method is public static PTransform<PCollectionTuple, PCollection<BeamSqlRow>> query(String sqlQuery), which relies on the named TupleTag to specify table name.
public static PTransform<PCollection<BeamSqlRow>, PCollection<BeamSqlRow>> simpleQuery(String sqlQuery) is a special case which runs on single table/PCollection. There's no exiting method to name it, so the table name in query is take as granted.
For both methods, potentially the table name would be mixed up, that's why I said a further task is needed to have a separated schema namespace for each query.
@lukecwik any comments?
There was a problem hiding this comment.
@lukecwik do you mean pcollection.simpleQuery('SELECT C1, C2'), to replace pcollection.simpleQuery('SELECT C1, C2 FROM TABLE_NAME')?
There was a problem hiding this comment.
@xumingmin Having the ability to use 'SELECT C1, C2 FROM TABLE_NAME' or 'SELECT C1, C2' doesn't matter to me. What matters to me is that they don't need to use a PCollectionTuple with a single PCollection and TupleTag pair.
I was thinking that the whole idea of global registration whenever someone calls query or simpleQuery will impact the users future pipeline construction depending on the order in which they apply parts of their pipeline is not a good idea. I can see how its useful in a CLI where they aren't building a pipeline programmatically but should be limited to paths which the CLI code handles.
There was a problem hiding this comment.
Agree, a global schema namespace doesn't sound good in DSL, would address the issue in a new task.
|
LGTM |
|
rebase to fix conflict. |
|
I don't think you want to register tables with BeamSqlEnv within expand(). BeamSqlEnv is currently the global namespace. |
|
As discussed above, there should be a BeamSqlEnv per query, to limit the scope of resisted tables. I would prefer to do it in another task, as it impacts both BeamSql and BeamSqlCli. |
|
SGTM |
|
Thanks @lukecwik, @xumingming Created BEAM2446 to track the next step. |
R: @takidau @xumingming
CC: @lukecwik