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

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

                Author: ASF GitHub Bot
            Created on: 06/Feb/19 18:53
            Start Date: 06/Feb/19 18:53
    Worklog Time Spent: 10m 
      Work Description: amaliujia commented on pull request #7745: [BEAM-5644] 
make Planner configurable
URL: https://github.com/apache/beam/pull/7745#discussion_r254403289
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
 ##########
 @@ -73,83 +70,18 @@ public static BeamSqlEnv inMemory(TableProvider... 
tableProviders) {
     return withTableProvider(inMemoryMetaStore);
   }
 
-  private void registerBuiltinUdf(Map<String, List<Method>> methods) {
-    for (Map.Entry<String, List<Method>> entry : methods.entrySet()) {
-      for (Method method : entry.getValue()) {
-        connection.getCurrentSchemaPlus().add(entry.getKey(), 
UdfImpl.create(method));
-      }
-    }
-  }
-
-  public void addSchema(String name, TableProvider tableProvider) {
-    connection.setSchema(name, tableProvider);
-  }
-
-  public void setCurrentSchema(String name) {
-    try {
-      connection.setSchema(name);
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** Register a UDF function which can be used in SQL expression. */
-  public void registerUdf(String functionName, Class<?> clazz, String method) {
-    connection.getCurrentSchemaPlus().add(functionName, UdfImpl.create(clazz, 
method));
-  }
-
-  /** Register a UDF function which can be used in SQL expression. */
-  public void registerUdf(String functionName, Class<? extends BeamSqlUdf> 
clazz) {
-    registerUdf(functionName, clazz, BeamSqlUdf.UDF_METHOD);
-  }
-
-  /**
-   * Register {@link SerializableFunction} as a UDF function which can be used 
in SQL expression.
-   * Note, {@link SerializableFunction} must have a constructor without 
arguments.
-   */
-  public void registerUdf(String functionName, SerializableFunction sfn) {
-    registerUdf(functionName, sfn.getClass(), "apply");
-  }
-
-  /**
-   * Register a UDAF function which can be used in GROUP-BY expression. See 
{@link
-   * org.apache.beam.sdk.transforms.Combine.CombineFn} on how to implement a 
UDAF.
-   */
-  public void registerUdaf(String functionName, Combine.CombineFn combineFn) {
-    connection.getCurrentSchemaPlus().add(functionName, new 
UdafImpl(combineFn));
-  }
-
-  /** Load all UDF/UDAF from {@link UdfUdafProvider}. */
-  public void loadUdfUdafFromProvider() {
-    ServiceLoader.<UdfUdafProvider>load(UdfUdafProvider.class)
-        .forEach(
-            ins -> {
-              ins.getBeamSqlUdfs().forEach((udfName, udfClass) -> 
registerUdf(udfName, udfClass));
-              ins.getSerializableFunctionUdfs()
-                  .forEach((udfName, udfFn) -> registerUdf(udfName, udfFn));
-              ins.getUdafs().forEach((udafName, udafFn) -> 
registerUdaf(udafName, udafFn));
-            });
-  }
-
-  public void loadBeamBuiltinFunctions() {
-    for (BeamBuiltinFunctionProvider provider :
-        ServiceLoader.load(BeamBuiltinFunctionProvider.class)) {
-      registerBuiltinUdf(provider.getBuiltinMethods());
-    }
-  }
-
   public BeamRelNode parseQuery(String query) throws ParseException {
     try {
       return planner.convertToBeamRel(query);
-    } catch (ValidationException | RelConversionException | SqlParseException 
e) {
+    } catch (Exception e) {
       throw new ParseException(String.format("Unable to parse query %s", 
query), e);
     }
   }
 
   public boolean isDdl(String sqlStatement) throws ParseException {
     try {
       return planner.parse(sqlStatement) instanceof SqlExecutableStatement;
-    } catch (SqlParseException e) {
+    } catch (Exception e) {
 
 Review comment:
   Right now it's less clear what exception should be throw from `QueryPlanner` 
interface, so I use a general `Exception` here. 
   
   I can change the exception back to what originally here it is by changing 
`QueryPlanner` interface. By doing so `QueryPlanner` interface can only be used 
for `CalciteQueryPlanner` because it throws Calcite's exceptions. I should add 
experimental annotation on it.
   
   Later when adding any new planners, we can gradually generalize the 
exception, probably starting from defining BeamSQL exceptions and let the 
interface throws that.
   
   Is there better option?  
 
----------------------------------------------------------------
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: 195305)
    Time Spent: 6h 50m  (was: 6h 40m)

> make Planner configurable 
> --------------------------
>
>                 Key: BEAM-5644
>                 URL: https://issues.apache.org/jira/browse/BEAM-5644
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Rui Wang
>            Assignee: Rui Wang
>            Priority: Major
>          Time Spent: 6h 50m
>  Remaining Estimate: 0h
>
> We can make planner configurable here: 
> [BeamQueryPlanner.java#L145|https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamQueryPlanner.java#L145]
>  
> By doing so, we can have different planner implementation to support 
> different SQL dialect.



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

Reply via email to