TheNeuralBit commented on a change in pull request #14273: URL: https://github.com/apache/beam/pull/14273#discussion_r605951248
########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapToKeys.java ########## @@ -0,0 +1,71 @@ +/* + * 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 org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@code MapToKeys} maps a {@code SerializableFunction} over keys of a {@code PCollection} of + * {@code KV<K1, V>}s and returns a {@code PCollection} of the keys. + * + * <p>Example of use: + * + * <pre>{@code + * PCollection<KV<Integer, String>> input = ...; + * PCollection<Double> output = + * input.apply(MapToKeys.via(Integer::doubleValue)); + * }</pre> + * + * <p>See also {@link MapToValues}. + * + * @param <K1> the type of the keys in the input {@code PCollection} + * @param <K2> the type of the elements in the output {@code PCollection} + */ +public class MapToKeys<K1, K2> Review comment: I'm not sure about `MapToKeys` and `MapToValues`, this can already be accomplished pretty concisely with: ``` pc.apply(Keys.create()) .apply(MapElements.into(type).via(func)) ``` ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapKeys.java ########## @@ -0,0 +1,71 @@ +/* + * 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 org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@code MapKeys} maps a {@code SerializableFunction} over keys of a {@code PCollection} of {@code + * KV<K1, V>}s and returns a {@code PCollection} of {@code KV<K2, V>}s. + * + * <p>Example of use: + * + * <pre>{@code + * PCollection<KV<Integer, String>> input = ...; + * PCollection<KV<Double, String> output = + * input.apply(MapKeys.via(Integer::doubleValue)); + * }</pre> + * + * <p>See also {@link MapValues}. + * + * @param <K1> the type of the keys in the input {@code PCollection} + * @param <K2> the type of the keys in the output {@code PCollection} + */ +public class MapKeys<K1, K2, V> extends PTransform<PCollection<KV<K1, V>>, PCollection<KV<K2, V>>> { + + private final SerializableFunction<K1, K2> fn; + + /** + * Returns a {@code MapKeys<K1, K2, V>} {@code PTransform}. + * + * @param <K1> the type of the keys in the input {@code PCollection} + * @param <K2> the type of the keys in the output {@code PCollection} + * @param <V> the type of the values in the input and output {@code PCollection}s + */ + public static <K1, K2, V> MapKeys<K1, K2, V> via(SerializableFunction<K1, K2> fn) { + return new MapKeys<>(fn); + } + + private MapKeys(SerializableFunction<K1, K2> fn) { + this.fn = fn; + } + + @Override + public PCollection<KV<K2, V>> expand(PCollection<KV<K1, V>> input) { + return input.apply( + "MapKeys", + MapElements.via( Review comment: It would be nice if these transforms provided all the same conveniences as [`MapElements`](https://beam.apache.org/releases/javadoc/2.28.0/org/apache/beam/sdk/transforms/MapElements.html), such as specifying the output type with `into` and handling exceptions with `exceptionsVia`, what do you think? It may be possible to accomplish that with inheritance. ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapKeys.java ########## @@ -0,0 +1,71 @@ +/* + * 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 org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@code MapKeys} maps a {@code SerializableFunction} over keys of a {@code PCollection} of {@code + * KV<K1, V>}s and returns a {@code PCollection} of {@code KV<K2, V>}s. Review comment: nit: I'd just make this docstring embrace the generics syntax ```suggestion * {@code MapKeys} maps a {@code SerializableFunction<K1,K2>} over keys of a {@code PCollection<KV<K1,V>>} and returns a {@code PCollection<KV<K2, V>>}s. ``` -- 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]
