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

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

                Author: ASF GitHub Bot
            Created on: 08/Oct/18 22:16
            Start Date: 08/Oct/18 22:16
    Worklog Time Spent: 10m 
      Work Description: XuMingmin closed pull request #6580: [BEAM-5202] 
register UDF/UDAF with ServiceLoader
URL: https://github.com/apache/beam/pull/6580
 
 
   

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/BeamSqlCli.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
index eb32d2a9a36..6d2c167acf5 100644
--- 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
@@ -34,8 +34,15 @@
   private MetaStore metaStore;
 
   public BeamSqlCli metaStore(MetaStore metaStore) {
+    return metaStore(metaStore, false);
+  }
+
+  public BeamSqlCli metaStore(MetaStore metaStore, boolean autoLoadUdfUdaf) {
     this.metaStore = metaStore;
     this.env = BeamSqlEnv.withTableProvider(metaStore);
+    if (autoLoadUdfUdaf) {
+      env.loadUdfUdafFromProvider();
+    }
 
     return this;
   }
diff --git 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
index ac9a6a17eba..2997db22728 100644
--- 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
@@ -85,11 +85,16 @@
 
   abstract List<UdafDefinition> udafDefinitions();
 
+  abstract boolean autoUdfUdafLoad();
+
   @Override
   public PCollection<Row> expand(PInput input) {
     BeamSqlEnv sqlEnv = BeamSqlEnv.readOnly(PCOLLECTION_NAME, 
toTableMap(input));
 
     registerFunctions(sqlEnv);
+    if (autoUdfUdafLoad()) {
+      sqlEnv.loadUdfUdafFromProvider();
+    }
 
     return BeamSqlRelUtils.toPCollection(input.getPipeline(), 
sqlEnv.parseQuery(queryString()));
   }
@@ -148,9 +153,13 @@ public static SqlTransform query(String queryString) {
         .setQueryString(queryString)
         .setUdafDefinitions(Collections.emptyList())
         .setUdfDefinitions(Collections.emptyList())
+        .setAutoUdfUdafLoad(false)
         .build();
   }
 
+  public SqlTransform withAutoUdfUdafLoad(boolean autoUdfUdafLoad) {
+    return toBuilder().setAutoUdfUdafLoad(autoUdfUdafLoad).build();
+  }
   /**
    * register a UDF function used in this query.
    *
@@ -203,6 +212,8 @@ static Builder builder() {
 
     abstract Builder setUdafDefinitions(List<UdafDefinition> udafDefinitions);
 
+    abstract Builder setAutoUdfUdafLoad(boolean autoUdfUdafLoad);
+
     abstract SqlTransform build();
   }
 
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 44eed399130..093961c59d2 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
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.extensions.sql.impl;
 
 import java.util.Map;
+import java.util.ServiceLoader;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.BeamSqlTable;
@@ -25,6 +26,7 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
+import org.apache.beam.sdk.extensions.sql.meta.provider.UdfUdafProvider;
 import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -99,6 +101,18 @@ public void registerUdaf(String functionName, 
Combine.CombineFn combineFn) {
     defaultSchema.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 BeamRelNode parseQuery(String query) throws ParseException {
     try {
       return planner.convertToBeamRel(query);
diff --git 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java
new file mode 100644
index 00000000000..40e3abbb1ab
--- /dev/null
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java
@@ -0,0 +1,41 @@
+/*
+ * 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.meta.provider;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.BeamSqlUdf;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+/** Provider for UDF and UDAF. */
+public interface UdfUdafProvider {
+  /** For UDFs implement {@link BeamSqlUdf}. */
+  default Map<String, Class<? extends BeamSqlUdf>> getBeamSqlUdfs() {
+    return Collections.emptyMap();
+  }
+
+  /** For UDFs implement {@link SerializableFunction}. */
+  default Map<String, SerializableFunction<?, ?>> 
getSerializableFunctionUdfs() {
+    return Collections.emptyMap();
+  }
+
+  default Map<String, Combine.CombineFn> getUdafs() {
+    return Collections.emptyMap();
+  }
+}
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 314bb1450ee..5b23151e795 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
@@ -20,7 +20,11 @@
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
 
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
 import org.apache.beam.sdk.extensions.sql.impl.ParseException;
+import org.apache.beam.sdk.extensions.sql.meta.provider.UdfUdafProvider;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
@@ -133,6 +137,39 @@ public void testUdf() throws Exception {
     pipeline.run().waitUntilFinish();
   }
 
+  /** test auto-provider UDF/UDAF. */
+  @Test
+  public void testAutoUdfUdaf() throws Exception {
+    Schema resultType =
+        
Schema.builder().addInt32Field("f_int2").addInt32Field("autoload_squarecubicsum").build();
+
+    Row row = Row.withSchema(resultType).addValues(0, 4890).build();
+
+    String sql =
+        "SELECT f_int2, autoload_squaresum(autoload_cubic(f_int)) AS 
`autoload_squarecubicsum`"
+            + " FROM PCOLLECTION GROUP BY f_int2";
+    PCollection<Row> result =
+        boundedInput1.apply("testUdaf", 
SqlTransform.query(sql).withAutoUdfUdafLoad(true));
+
+    PAssert.that(result).containsInAnyOrder(row);
+    pipeline.run().waitUntilFinish();
+  }
+
+  /** Auto provider for test. */
+  @AutoService(UdfUdafProvider.class)
+  public static class UdfUdafProviderTest implements UdfUdafProvider {
+
+    @Override
+    public Map<String, Class<? extends BeamSqlUdf>> getBeamSqlUdfs() {
+      return ImmutableMap.of("autoload_cubic", CubicInteger.class);
+    }
+
+    @Override
+    public Map<String, CombineFn> getUdafs() {
+      return ImmutableMap.of("autoload_squaresum", new SquareSum());
+    }
+  }
+
   /** UDAF(CombineFn) for test, which returns the sum of square. */
   public static class SquareSum extends CombineFn<Integer, Integer, Integer> {
     @Override


 

----------------------------------------------------------------
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: 152441)
    Time Spent: 3h 10m  (was: 3h)

> register UDF/UDAF with ServiceLoader
> ------------------------------------
>
>                 Key: BEAM-5202
>                 URL: https://issues.apache.org/jira/browse/BEAM-5202
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Xu Mingmin
>            Assignee: Xu Mingmin
>            Priority: Major
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> create a ServiceLoader for UDF/UDAF register, if the autoLoad option is set 
> to TRUE.



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

Reply via email to