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

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

                Author: ASF GitHub Bot
            Created on: 31/Oct/18 20:40
            Start Date: 31/Oct/18 20:40
    Worklog Time Spent: 10m 
      Work Description: amaliujia closed pull request #6865: [BEAM-5892] 
Improve UDF registration
URL: https://github.com/apache/beam/pull/6865
 
 
   

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/BeamSqlEnv.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
index 093961c59d2..6e305b21094 100644
--- 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl;
 
+import java.lang.reflect.Method;
 import java.util.Map;
 import java.util.ServiceLoader;
 import org.apache.beam.sdk.annotations.Experimental;
@@ -77,7 +78,12 @@ public static BeamSqlEnv inMemory(TableProvider... 
tableProviders) {
 
   /** Register a UDF function which can be used in SQL expression. */
   public void registerUdf(String functionName, Class<?> clazz, String method) {
-    defaultSchema.add(functionName, ScalarFunctionImpl.create(clazz, method));
+    Method[] methods = clazz.getMethods();
+    for (int i = 0; i < method.length(); i++) {
+      if (methods[i].getName().equals(method) && !methods[i].isBridge()) {
+        defaultSchema.add(functionName, ScalarFunctionImpl.create(methods[i]));
+      }
+    }
   }
 
   /** Register a UDF function which can be used in SQL expression. */
diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
index 5b23151e795..8538d62591b 100644
--- 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
+++ 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
@@ -108,12 +108,17 @@ public void testRawCombineFnSubclass() {
   @Test
   public void testUdf() throws Exception {
     Schema resultType = 
Schema.builder().addInt32Field("f_int").addInt32Field("cubicvalue").build();
+    Schema resultType2 =
+        
Schema.builder().addInt64Field("f_long").addInt64Field("cubicLongValue").build();
+    Schema resultType3 =
+        
Schema.builder().addDoubleField("f_double").addDoubleField("cubicDoubleValue").build();
+
     Row row = Row.withSchema(resultType).addValues(2, 8).build();
 
     String sql1 = "SELECT f_int, cubic1(f_int) as cubicvalue FROM PCOLLECTION 
WHERE f_int = 2";
     PCollection<Row> result1 =
         boundedInput1.apply(
-            "testUdf1", SqlTransform.query(sql1).registerUdf("cubic1", 
CubicInteger.class));
+            "testUdf1", SqlTransform.query(sql1).registerUdf("cubic1", 
Cubic.class));
     PAssert.that(result1).containsInAnyOrder(row);
 
     String sql2 = "SELECT f_int, cubic2(f_int) as cubicvalue FROM PCOLLECTION 
WHERE f_int = 2";
@@ -134,6 +139,25 @@ public void testUdf() throws Exception {
     Row subStrRow = Row.withSchema(subStrSchema).addValues(2, "s").build();
     PAssert.that(result3).containsInAnyOrder(subStrRow);
 
+    String sql4 =
+        "SELECT f_long, cubic1(f_long) as cubicLongValue FROM PCOLLECTION 
WHERE f_long = 1000";
+    PCollection<Row> result4 =
+        PCollectionTuple.of(new TupleTag<>("PCOLLECTION"), boundedInput1)
+            .apply("testUdf4", SqlTransform.query(sql4).registerUdf("cubic1", 
Cubic.class));
+
+    Row longRow = Row.withSchema(resultType2).addValues(1000L, 1000L * 1000L * 
1000L).build();
+    PAssert.that(result4).containsInAnyOrder(longRow);
+
+    String sql5 =
+        "SELECT f_double, cubic1(f_double) as cubicDoubleValue FROM 
PCOLLECTION WHERE f_double = 1.0";
+    PCollection<Row> result5 =
+        PCollectionTuple.of(new TupleTag<>("PCOLLECTION"), boundedInput1)
+            .apply("testUdf5", SqlTransform.query(sql5).registerUdf("cubic1", 
Cubic.class));
+
+    Row doubleRow = Row.withSchema(resultType3).addValues(1.0d, 1.0d * 1.0d * 
1.0d).build();
+
+    PAssert.that(result5).containsInAnyOrder(doubleRow);
+
     pipeline.run().waitUntilFinish();
   }
 
@@ -161,7 +185,7 @@ public void testAutoUdfUdaf() throws Exception {
 
     @Override
     public Map<String, Class<? extends BeamSqlUdf>> getBeamSqlUdfs() {
-      return ImmutableMap.of("autoload_cubic", CubicInteger.class);
+      return ImmutableMap.of("autoload_cubic", Cubic.class);
     }
 
     @Override
@@ -233,10 +257,18 @@ public Integer addInput(Integer accumulator, Integer 
input) {
   }
 
   /** An example UDF for test. */
-  public static class CubicInteger implements BeamSqlUdf {
+  public static class Cubic implements BeamSqlUdf {
     public static Integer eval(Integer input) {
       return input * input * input;
     }
+
+    public static Long eval(Long input) {
+      return input * input * input;
+    }
+
+    public static Double eval(Double input) {
+      return input * input * input;
+    }
   }
 
   /** An example UDF with {@link SerializableFunction}. */


 

----------------------------------------------------------------
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: 161309)
    Time Spent: 50m  (was: 40m)

> Allow registering UDF with the same method name but different argument list
> ---------------------------------------------------------------------------
>
>                 Key: BEAM-5892
>                 URL: https://issues.apache.org/jira/browse/BEAM-5892
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Rui Wang
>            Assignee: Rui Wang
>            Priority: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>




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

Reply via email to