[ 
https://issues.apache.org/jira/browse/BEAM-4562?focusedWorklogId=123210&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-123210
 ]

ASF GitHub Bot logged work on BEAM-4562:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 14/Jul/18 05:57
            Start Date: 14/Jul/18 05:57
    Worklog Time Spent: 10m 
      Work Description: XuMingmin closed pull request #5930: [BEAM-4562] [SQL] 
Apply JDBC rules globally
URL: https://github.com/apache/beam/pull/5930
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
index 4b66e93d514..8d54ed09613 100644
--- 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
@@ -81,7 +81,7 @@
       Thread.currentThread().setContextClassLoader(origLoader);
     }
     // inject beam rules into planner
-    Hook.PLANNER.addThread(
+    Hook.PLANNER.add(
         new Function<RelOptPlanner, Void>() {
           @Override
           public Void apply(RelOptPlanner planner) {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 123210)
    Time Spent: 2h  (was: 1h 50m)

> [SQL] Fix INSERT VALUES in JdbcDriver 
> --------------------------------------
>
>                 Key: BEAM-4562
>                 URL: https://issues.apache.org/jira/browse/BEAM-4562
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Anton Kedin
>            Assignee: Andrew Pilloud
>            Priority: Major
>          Time Spent: 2h
>  Remaining Estimate: 0h
>
> Executing INSERT VALUES against JdbcDriver fails. Executing similar 
> statements against BeamSqlEnv works fine. Example:
> {code:java}
>     TestTableProvider tableProvider = new TestTableProvider();
>     Connection connection = JdbcDriver.connect(tableProvider);
>     connection
>         .createStatement()
>         .executeUpdate("CREATE TABLE person (id BIGINT, name VARCHAR) TYPE 
> 'test'");
>     connection.createStatement().executeQuery("INSERT INTO person VALUES(3, 
> 'yyy')");
> {code}
>  Output:
> {code}
> java.sql.SQLException: Error while executing SQL "INSERT INTO person 
> VALUES(3, 'yyy')": Node [rel#9:Subset#1.ENUMERABLE.[]] could not be 
> implemented; planner state:
> Root: rel#9:Subset#1.ENUMERABLE.[]
> Original rel:
> BeamIOSinkRel(subset=[rel#9:Subset#1.ENUMERABLE.[]], table=[[beam, person]], 
> operation=[INSERT], flattened=[false]): rowcount = 1.0, cumulative cost = 
> {1.0 rows, 0.0 cpu, 0.0 io}, id = 6
>   LogicalValues(subset=[rel#5:Subset#0.NONE.[]], tuples=[[{ 3, 'yyy' }]]): 
> rowcount = 1.0, cumulative cost = {1.0 rows, 1.0 cpu, 0.0 io}, id = 0
> Sets:
> Set#0, type: RecordType(BIGINT id, VARCHAR name)
>       rel#5:Subset#0.NONE.[], best=null, importance=0.81
>               rel#0:LogicalValues.NONE.[[0, 1], [1]](type=RecordType(BIGINT 
> id, VARCHAR name),tuples=[{ 3, 'yyy' }]), rowcount=1.0, cumulative cost={inf}
>       rel#14:Subset#0.BEAM_LOGICAL.[], best=null, importance=0.81
>       rel#20:Subset#0.ENUMERABLE.[], best=rel#19, importance=0.405
>               rel#19:EnumerableValues.ENUMERABLE.[[0, 1], 
> [1]](type=RecordType(BIGINT id, VARCHAR name),tuples=[{ 3, 'yyy' }]), 
> rowcount=1.0, cumulative cost={1.0 rows, 1.0 cpu, 0.0 io}
> Set#1, type: RecordType(BIGINT ROWCOUNT)
>       rel#7:Subset#1.BEAM_LOGICAL.[], best=null, importance=0.9
>               
> rel#6:BeamIOSinkRel.BEAM_LOGICAL.[](input=rel#5:Subset#0.NONE.[],table=[beam, 
> person],operation=INSERT,flattened=false), rowcount=1.0, cumulative cost={inf}
>               
> rel#15:BeamIOSinkRel.BEAM_LOGICAL.[](input=rel#14:Subset#0.BEAM_LOGICAL.[],table=[beam,
>  person],operation=INSERT,flattened=false), rowcount=1.0, cumulative 
> cost={inf}
>       rel#9:Subset#1.ENUMERABLE.[], best=null, importance=1.0
>               
> rel#10:AbstractConverter.ENUMERABLE.[](input=rel#7:Subset#1.BEAM_LOGICAL.[],convention=ENUMERABLE,sort=[]),
>  rowcount=1.0, cumulative cost={inf}
>               
> rel#11:BeamEnumerableConverter.ENUMERABLE.[](input=rel#7:Subset#1.BEAM_LOGICAL.[]),
>  rowcount=1.0, cumulative cost={inf}{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to