You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/02/18 16:54:08 UTC

[GitHub] [beam] kennknowles commented on a change in pull request #14014: Add AutoLoadedSqlTransform, which runs SQL queries using the auto loading mechanisms.

kennknowles commented on a change in pull request #14014:
URL: https://github.com/apache/beam/pull/14014#discussion_r578582239



##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/AutoLoadedSqlTransform.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.ServiceLoader;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
+import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
+import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
+import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * A PTransform for executing a query from a list of DDL strings and a query string.
+ *
+ * <p>It loads all registered TableProviders, builtin functions and UDFs.
+ */
+@Experimental
+public class AutoLoadedSqlTransform extends PTransform<PBegin, PDone> {
+  private final List<String> ddls;
+  private final String query;
+
+  public AutoLoadedSqlTransform(List<String> ddls, String query) {
+    this.ddls = ddls;
+    this.query = query;
+  }
+
+  @Override
+  public PDone expand(PBegin input) {
+    MetaStore store = new InMemoryMetaStore();
+    ServiceLoader.load(TableProvider.class).forEach(store::registerProvider);
+    BeamSqlEnv env =
+        BeamSqlEnv.builder(store)
+            .autoLoadBuiltinFunctions()
+            .autoLoadUserDefinedFunctions()
+            .setPipelineOptions(input.getPipeline().getOptions())
+            .build();
+    ddls.forEach(env::executeDdl);
+    BeamSqlRelUtils.toPCollection(input.getPipeline(), env.parseQuery(query));

Review comment:
       If we do keep it as a separate transform, I think the best thing to do here would be to take the things that were loaded via service loader and pass them to builder methods on `SqlTransform`.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
##########
@@ -128,12 +127,12 @@
       sqlEnvBuilder.autoLoadUserDefinedFunctions();
     }
 
-    sqlEnvBuilder.setQueryPlannerClassName(
-        MoreObjects.firstNonNull(
-            queryPlannerClassName(),
-            input.getPipeline().getOptions().as(BeamSqlPipelineOptions.class).getPlannerName()));
-
-    sqlEnvBuilder.setPipelineOptions(input.getPipeline().getOptions());
+    BeamSqlPipelineOptions options =
+        input.getPipeline().getOptions().as(BeamSqlPipelineOptions.class);
+    if (queryPlannerClassName() != null) {
+      options.setPlannerName(queryPlannerClassName());

Review comment:
       This method should not mutate the options.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/AutoLoadedSqlTransform.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.ServiceLoader;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
+import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
+import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
+import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PDone;
+
+/**
+ * A PTransform for executing a query from a list of DDL strings and a query string.
+ *
+ * <p>It loads all registered TableProviders, builtin functions and UDFs.
+ */
+@Experimental
+public class AutoLoadedSqlTransform extends PTransform<PBegin, PDone> {

Review comment:
       I think it makes sense to just make this part of `SqlTransform`. It could be either default with opt-out (`.withoutAutoLoading`) or non-default with opt-in (`.withAutoLoading`)

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
##########
@@ -225,13 +225,8 @@ public BeamSqlEnvBuilder autoLoadBuiltinFunctions() {
       return this;
     }
 
-    public BeamSqlEnvBuilder setQueryPlannerClassName(String name) {

Review comment:
       I see that eliminating this method removes a potential source of inconsistency where the options say one planner and the method says another.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org