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

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

                Author: ASF GitHub Bot
            Created on: 30/Oct/18 20:58
            Start Date: 30/Oct/18 20:58
    Worklog Time Spent: 10m 
      Work Description: jklukas commented on a change in pull request #6892: 
[BEAM-5922] Refactor Map, FlatMap, and Filter on MapperBase
URL: https://github.com/apache/beam/pull/6892#discussion_r229483690
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapperBase.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.transforms;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.transforms.Contextful.Fn;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Abstract class providing a base for {@link PTransform}s that map a simple 
function over elements.
+ *
+ * <p>The passed {@code fn} returns an {@link Iterable}, each element of which 
will be sent to
+ * output. That interface naturally expresses a FlatMap, but both Map (always 
one output element)
+ * and Filter (either the input element is passed through or an empty 
collection) can both be
+ * expressed as special cases.
+ */
+abstract class MapperBase<InputT, OutputT>
+    extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
+  @Nullable private final transient TypeDescriptor<InputT> inputType;
+  @Nullable final transient TypeDescriptor<OutputT> outputType;
+  @Nullable private final transient Object originalFnForDisplayData;
+  @Nullable private final Contextful<Fn<InputT, Iterable<OutputT>>> fn;
+
+  MapperBase(
+      @Nullable String name,
+      @Nullable Contextful<Fn<InputT, Iterable<OutputT>>> fn,
+      @Nullable Object originalFnForDisplayData,
+      @Nullable TypeDescriptor<InputT> inputType,
+      TypeDescriptor<OutputT> outputType) {
+    super(name);
 
 Review comment:
   Previously, MapElements and FlatMapElements called `apply(name, transform)` 
but Filter omitted the name. Browsing through the DisplayData code, it looks 
that passing `name` to the `PTransform` constructor here has the same effect as 
passing a name to `apply`.
   
   If the naming here ends up being different from previous versions, will that 
cause issues with state if a pipeline is redeployed with an updated version of 
the Beam API? If this is critical, I can do more investigation into how to test 
this. Currently, it looks like a tricky thing to test since the DoFn is created 
internally and isn't available to the class.

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

> Common base class for FlatMapElements, MapElements, and Filter
> --------------------------------------------------------------
>
>                 Key: BEAM-5922
>                 URL: https://issues.apache.org/jira/browse/BEAM-5922
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-core
>            Reporter: Jeff Klukas
>            Assignee: Jeff Klukas
>            Priority: Minor
>              Labels: pull-request-available
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> By having FlatMapElements, MapElements, and Filter all inherit from a common 
> MapperBase, we can reduce code duplication between these three classes for 
> better consistency in future changes. There should be no change in 
> functionality or public interface with this change.
> In particular, this change will simplify the implementation of failure 
> handling
> across these transforms in https://issues.apache.org/jira/browse/BEAM-5638



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

Reply via email to