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/12/06 19:22:08 UTC

[GitHub] [beam] apilloud commented on a change in pull request #16113: [BEAM-12976] Implement pipeline visitor to get global field access in…

apilloud commented on a change in pull request #16113:
URL: https://github.com/apache/beam/pull/16113#discussion_r763294590



##########
File path: runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.runners.core.construction.graph;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+/** Computes which Schema fields are (or conversely, are not) accessed in a pipeline. */
+class FieldAccessVisitor extends PipelineVisitor.Defaults {
+  private final Map<String, FieldAccessDescriptor> pCollectionFieldAccess = new HashMap<>();
+
+  /**
+   * Returns a map keyed by the {@link TupleTag} ID referencing a PCollection. Values are the set of
+   * fields accessed on that PCollection.
+   */
+  ImmutableMap<String, FieldAccessDescriptor> getPCollectionFieldAccess() {
+    return ImmutableMap.copyOf(pCollectionFieldAccess);
+  }
+
+  @Override
+  public void visitPrimitiveTransform(Node node) {
+    Map<String, FieldAccessDescriptor> currentFieldAccess = getFieldAccess(node);
+    for (Entry<String, FieldAccessDescriptor> entry : currentFieldAccess.entrySet()) {
+      FieldAccessDescriptor previousFieldAccess = pCollectionFieldAccess.get(entry.getKey());
+      FieldAccessDescriptor newFieldAccess =
+          previousFieldAccess == null
+              ? entry.getValue()
+              : FieldAccessDescriptor.union(
+                  ImmutableList.of(previousFieldAccess, entry.getValue()));
+      pCollectionFieldAccess.put(entry.getKey(), newFieldAccess);
+    }
+  }
+
+  private static Map<String, FieldAccessDescriptor> getFieldAccess(Node node) {
+    PTransform<?, ?> transform = node.getTransform();
+    HashMap<String, FieldAccessDescriptor> access = new HashMap<>();
+
+    if (transform instanceof MultiOutput) {
+      DoFn<?, ?> fn = ((MultiOutput<?, ?>) transform).getFn();
+      Pair<TupleTag<?>, PCollection<?>> mainInput = getMainInputTagId(node);
+      FieldAccessDescriptor fields =
+          ParDo.getDoFnSchemaInformation(fn, mainInput.getRight()).getFieldAccessDescriptor();
+      access.put(mainInput.getLeft().getId(), fields);
+    }
+
+    // For every input without field access info, we must assume all fields need to be accessed.
+    for (TupleTag<?> tag : node.getInputs().keySet()) {
+      if (!access.containsKey(tag.getId())) {
+        access.put(tag.getId(), FieldAccessDescriptor.withAllFields());
+      }
+    }
+
+    return ImmutableMap.copyOf(access);
+  }
+
+  private static Pair<TupleTag<?>, PCollection<?>> getMainInputTagId(Node node) {

Review comment:
       nit: This is also deceptively named, it appears to actually be `getMainInput`. (It gets both the PCollection and tagid.)

##########
File path: runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.runners.core.construction.graph;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+/** Computes which Schema fields are (or conversely, are not) accessed in a pipeline. */
+class FieldAccessVisitor extends PipelineVisitor.Defaults {
+  private final Map<String, FieldAccessDescriptor> pCollectionFieldAccess = new HashMap<>();
+
+  /**
+   * Returns a map keyed by the {@link TupleTag} ID referencing a PCollection. Values are the set of
+   * fields accessed on that PCollection.
+   */
+  ImmutableMap<String, FieldAccessDescriptor> getPCollectionFieldAccess() {
+    return ImmutableMap.copyOf(pCollectionFieldAccess);
+  }
+
+  @Override
+  public void visitPrimitiveTransform(Node node) {
+    Map<String, FieldAccessDescriptor> currentFieldAccess = getFieldAccess(node);
+    for (Entry<String, FieldAccessDescriptor> entry : currentFieldAccess.entrySet()) {
+      FieldAccessDescriptor previousFieldAccess = pCollectionFieldAccess.get(entry.getKey());
+      FieldAccessDescriptor newFieldAccess =
+          previousFieldAccess == null
+              ? entry.getValue()
+              : FieldAccessDescriptor.union(
+                  ImmutableList.of(previousFieldAccess, entry.getValue()));
+      pCollectionFieldAccess.put(entry.getKey(), newFieldAccess);
+    }
+  }
+
+  private static Map<String, FieldAccessDescriptor> getFieldAccess(Node node) {
+    PTransform<?, ?> transform = node.getTransform();
+    HashMap<String, FieldAccessDescriptor> access = new HashMap<>();
+
+    if (transform instanceof MultiOutput) {
+      DoFn<?, ?> fn = ((MultiOutput<?, ?>) transform).getFn();
+      Pair<TupleTag<?>, PCollection<?>> mainInput = getMainInputTagId(node);

Review comment:
       nit:  I believe this function reduces readability (especially with the `Pair` return type) it would be cleaner inline.

##########
File path: runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.runners.core.construction.graph;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+/** Computes which Schema fields are (or conversely, are not) accessed in a pipeline. */
+class FieldAccessVisitor extends PipelineVisitor.Defaults {
+  private final Map<String, FieldAccessDescriptor> pCollectionFieldAccess = new HashMap<>();
+
+  /**
+   * Returns a map keyed by the {@link TupleTag} ID referencing a PCollection. Values are the set of
+   * fields accessed on that PCollection.
+   */
+  ImmutableMap<String, FieldAccessDescriptor> getPCollectionFieldAccess() {
+    return ImmutableMap.copyOf(pCollectionFieldAccess);
+  }
+
+  @Override
+  public void visitPrimitiveTransform(Node node) {
+    Map<String, FieldAccessDescriptor> currentFieldAccess = getFieldAccess(node);
+    for (Entry<String, FieldAccessDescriptor> entry : currentFieldAccess.entrySet()) {
+      FieldAccessDescriptor previousFieldAccess = pCollectionFieldAccess.get(entry.getKey());
+      FieldAccessDescriptor newFieldAccess =
+          previousFieldAccess == null
+              ? entry.getValue()
+              : FieldAccessDescriptor.union(
+                  ImmutableList.of(previousFieldAccess, entry.getValue()));
+      pCollectionFieldAccess.put(entry.getKey(), newFieldAccess);
+    }
+  }
+
+  private static Map<String, FieldAccessDescriptor> getFieldAccess(Node node) {
+    PTransform<?, ?> transform = node.getTransform();
+    HashMap<String, FieldAccessDescriptor> access = new HashMap<>();
+
+    if (transform instanceof MultiOutput) {
+      DoFn<?, ?> fn = ((MultiOutput<?, ?>) transform).getFn();
+      Pair<TupleTag<?>, PCollection<?>> mainInput = getMainInputTagId(node);
+      FieldAccessDescriptor fields =
+          ParDo.getDoFnSchemaInformation(fn, mainInput.getRight()).getFieldAccessDescriptor();
+      access.put(mainInput.getLeft().getId(), fields);
+    }
+
+    // For every input without field access info, we must assume all fields need to be accessed.
+    for (TupleTag<?> tag : node.getInputs().keySet()) {
+      if (!access.containsKey(tag.getId())) {
+        access.put(tag.getId(), FieldAccessDescriptor.withAllFields());
+      }
+    }
+
+    return ImmutableMap.copyOf(access);
+  }
+
+  private static Pair<TupleTag<?>, PCollection<?>> getMainInputTagId(Node node) {
+    HashSet<TupleTag<?>> mainInputTags = new HashSet<>(node.getInputs().keySet());

Review comment:
       nit: This is deceptively named, isn't this just `inputTags`?

##########
File path: runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.runners.core.construction.graph;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+/** Computes which Schema fields are (or conversely, are not) accessed in a pipeline. */
+class FieldAccessVisitor extends PipelineVisitor.Defaults {
+  private final Map<String, FieldAccessDescriptor> pCollectionFieldAccess = new HashMap<>();
+
+  /**
+   * Returns a map keyed by the {@link TupleTag} ID referencing a PCollection. Values are the set of
+   * fields accessed on that PCollection.
+   */
+  ImmutableMap<String, FieldAccessDescriptor> getPCollectionFieldAccess() {
+    return ImmutableMap.copyOf(pCollectionFieldAccess);
+  }
+
+  @Override
+  public void visitPrimitiveTransform(Node node) {
+    Map<String, FieldAccessDescriptor> currentFieldAccess = getFieldAccess(node);
+    for (Entry<String, FieldAccessDescriptor> entry : currentFieldAccess.entrySet()) {
+      FieldAccessDescriptor previousFieldAccess = pCollectionFieldAccess.get(entry.getKey());
+      FieldAccessDescriptor newFieldAccess =
+          previousFieldAccess == null
+              ? entry.getValue()
+              : FieldAccessDescriptor.union(
+                  ImmutableList.of(previousFieldAccess, entry.getValue()));
+      pCollectionFieldAccess.put(entry.getKey(), newFieldAccess);
+    }
+  }
+
+  private static Map<String, FieldAccessDescriptor> getFieldAccess(Node node) {
+    PTransform<?, ?> transform = node.getTransform();
+    HashMap<String, FieldAccessDescriptor> access = new HashMap<>();
+
+    if (transform instanceof MultiOutput) {
+      DoFn<?, ?> fn = ((MultiOutput<?, ?>) transform).getFn();
+      Pair<TupleTag<?>, PCollection<?>> mainInput = getMainInputTagId(node);
+      FieldAccessDescriptor fields =
+          ParDo.getDoFnSchemaInformation(fn, mainInput.getRight()).getFieldAccessDescriptor();

Review comment:
       How will this handle inputs that don't have schemas (and below for other inputs)?

##########
File path: runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FieldAccessVisitor.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.runners.core.construction.graph;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.MultiOutput;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+/** Computes which Schema fields are (or conversely, are not) accessed in a pipeline. */
+class FieldAccessVisitor extends PipelineVisitor.Defaults {
+  private final Map<String, FieldAccessDescriptor> pCollectionFieldAccess = new HashMap<>();

Review comment:
       Shouldn't this be `Map<PCollection, FieldAccessDescriptor>`?




-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

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