[
https://issues.apache.org/jira/browse/BEAM-621?focusedWorklogId=601319&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-601319
]
ASF GitHub Bot logged work on BEAM-621:
---------------------------------------
Author: ASF GitHub Bot
Created on: 24/May/21 18:14
Start Date: 24/May/21 18:14
Worklog Time Spent: 10m
Work Description: TheNeuralBit commented on a change in pull request
#14273:
URL: https://github.com/apache/beam/pull/14273#discussion_r638165582
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleMapWithFailures.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.transforms.Contextful.Fn;
+import org.apache.beam.sdk.transforms.WithFailures.ExceptionElement;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A {@code PTransform} that adds exception handling to {@link MapKeys} and
{@link MapValues} using
+ * {@link MapElements.MapWithFailures}.
+ */
+@SuppressWarnings({
+ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class SimpleMapWithFailures<InputT, OutputT, FailureT>
Review comment:
Does this need to be public, or could we make it package-private:
```suggestion
class SimpleMapWithFailures<InputT, OutputT, FailureT>
```
This looks like an internal class we don't want users to create directly.
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleMapWithFailures.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.transforms.Contextful.Fn;
+import org.apache.beam.sdk.transforms.WithFailures.ExceptionElement;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A {@code PTransform} that adds exception handling to {@link MapKeys} and
{@link MapValues} using
+ * {@link MapElements.MapWithFailures}.
+ */
+@SuppressWarnings({
+ "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
Review comment:
```suggestion
```
Please don't suppress the nullness check in new code (same goes for the
other new classes here). If there are issues with nullness you can debug them
locally by running `./gradlew compileJava -PenableCheckerFramework=true` (our
Java PreCommit does this by default as well)
--
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 601319)
Time Spent: 2h 10m (was: 2h)
> Add MapValues and MapKeys functions
> -----------------------------------
>
> Key: BEAM-621
> URL: https://issues.apache.org/jira/browse/BEAM-621
> Project: Beam
> Issue Type: New Feature
> Components: sdk-java-core, sdk-py-core
> Reporter: Jean-Baptiste Onofré
> Assignee: Vitaly Terentyev
> Priority: P3
> Labels: Clarified, newbie, starter
> Time Spent: 2h 10m
> Remaining Estimate: 0h
>
> Currently, we have the {{MapElements}} {{PTransform}} that "convert" a
> {{PCollection}} of {{KV}} to another {{PCollection}} (for instance
> {{String}}).
> A very classic mapping function is to just have the keys or values of {{KV}}.
> To do it currently, we can use {{MapElements}} or a generic {{ParDo}} (with
> {{DoFn}}).
> It would be helpful and reduce the user code to have {{MapValues}} and
> {{MapKeys}}. It would take a {{PCollection}} of {{KV}}: {{MapKeys}} will map
> the input {{PCollection}} to a {{PCollection}} of {{K}} and {{MapValues}} to
> a {{PCollection}} of {{V}}.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)