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

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

                Author: ASF GitHub Bot
            Created on: 01/Nov/18 19:18
            Start Date: 01/Nov/18 19:18
    Worklog Time Spent: 10m 
      Work Description: amaliujia commented on a change in pull request #6913: 
[BEAM-5921] [SQL] Support Joda types for UDF arguments
URL: https://github.com/apache/beam/pull/6913#discussion_r230158446
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
 ##########
 @@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.impl;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+import com.google.common.collect.ImmutableMultimap;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.linq4j.function.SemiStrict;
+import org.apache.calcite.linq4j.function.Strict;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ImplementableFunction;
+import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlOperatorBinding;
+
+/** Beam-customized version from {@link ScalarFunctionImpl}, to address 
BEAM-5921. */
+public class UdfImpl extends UdfImplReflectiveFunctionBase
+    implements ScalarFunction, ImplementableFunction {
+
+  private final CallImplementor implementor;
+
+  /** Private constructor. */
+  private UdfImpl(Method method, CallImplementor implementor) {
+    super(method);
+    this.implementor = implementor;
+  }
+
+  /** Creates {@link org.apache.calcite.schema.ScalarFunction} for each method 
in a given class. */
+  public static ImmutableMultimap<String, ScalarFunction> createAll(Class<?> 
clazz) {
+    final ImmutableMultimap.Builder<String, ScalarFunction> builder = 
ImmutableMultimap.builder();
+    for (Method method : clazz.getMethods()) {
+      if (method.getDeclaringClass() == Object.class) {
+        continue;
+      }
+      if (!Modifier.isStatic(method.getModifiers()) && 
!classHasPublicZeroArgsConstructor(clazz)) {
+        continue;
+      }
+      final ScalarFunction function = create(method);
+      builder.put(method.getName(), function);
+    }
+    return builder.build();
+  }
+
+  /**
+   * Creates {@link org.apache.calcite.schema.ScalarFunction} from given class.
+   *
+   * <p>If a method of the given name is not found or it does not suit, 
returns {@code null}.
+   *
+   * @param clazz class that is used to implement the function
+   * @param methodName Method name (typically "eval")
+   * @return created {@link ScalarFunction} or null
+   */
+  public static ScalarFunction create(Class<?> clazz, String methodName) {
 
 Review comment:
   For further improving our UDF registration, we can deprecate this function, 
which registers a UDF by class and method name, especially because of the fact 
that this function only registers one method of one class.
   
    All registration can be done in the same way as 
https://github.com/apache/beam/pull/6898.
   
   This will be another story and can be done in separate PRs.
   

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 161704)
    Time Spent: 1.5h  (was: 1h 20m)

> [SQL] Support Joda types for UDF arguments
> ------------------------------------------
>
>                 Key: BEAM-5921
>                 URL: https://issues.apache.org/jira/browse/BEAM-5921
>             Project: Beam
>          Issue Type: Bug
>          Components: dsl-sql
>            Reporter: Anton Kedin
>            Assignee: Xu Mingmin
>            Priority: Major
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> We call ScalarFunctionImpl.create() to register a UDF with Calcite schema in 
> BeamSqlEnv. Internally it uses Calcite's internal mapping 
> (JavaToSqlTypeConversionRules) to map Java types to SQL types to create a 
> function signature that gets registered in the schema. Problem is that this 
> logic is not extensible and doesn't include Joda types support (maybe others 
> as well).
> We can work around this by constructing our own subclass of Function that 
> gets registered in the schema instead of calling ScalarFunctionImpl.create(). 
> This logic can use our own custom mapping (or fall back to Calcite 
> implementation if needed).



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

Reply via email to