/** * Returns a new {@link MapElements} transform with the given type descriptor for the output type, * but the mapping function yet to be specified using {@link #via(SerializableFunction)}. */ public static <OutputT> MapElements<?, OutputT> into(final TypeDescriptor<OutputT> outputType) { return new MapElements<>(null, null, null, outputType); }
/** * For a {@code SerializableFunction<InputT, OutputT>} {@code fn} and output type descriptor, * returns a {@code PTransform} that takes an input {@code PCollection<InputT>} and returns a * {@code PCollection<OutputT>} containing {@code fn.apply(v)} for every element {@code v} in the * input. * * <p>Example of use in Java 8: * * <pre>{@code * PCollection<Integer> wordLengths = words.apply( * MapElements.into(TypeDescriptors.integers()) * .via((String word) -> word.length())); * }</pre> * * <p>In Java 7, the overload {@link #via(SimpleFunction)} is more concise as the output type * descriptor need not be provided. */ public <NewInputT> MapElements<NewInputT, OutputT> via( SerializableFunction<NewInputT, OutputT> fn) { return new MapElements<>(Contextful.fn(fn), fn, TypeDescriptors.inputOf(fn), outputType); }
/** * Like {@link #via(SerializableFunction)}, but supports access to context, such as side inputs. */ @Experimental(Kind.CONTEXTFUL) public <NewInputT> MapElements<NewInputT, OutputT> via(Contextful<Fn<NewInputT, OutputT>> fn) { return new MapElements<>( fn, fn.getClosure(), TypeDescriptors.inputOf(fn.getClosure()), outputType); }
/** * For a {@code SimpleFunction<InputT, OutputT>} {@code fn}, returns a {@code PTransform} that * takes an input {@code PCollection<InputT>} and returns a {@code PCollection<OutputT>} * containing {@code fn.apply(v)} for every element {@code v} in the input. * * <p>This overload is intended primarily for use in Java 7. In Java 8, the overload {@link * #via(SerializableFunction)} supports use of lambda for greater concision. * * <p>Example of use in Java 7: * * <pre>{@code * PCollection<String> words = ...; * PCollection<Integer> wordsPerLine = words.apply(MapElements.via( * new SimpleFunction<String, Integer>() { * public Integer apply(String word) { * return word.length(); * } * })); * }</pre> */ public static <InputT, OutputT> MapElements<InputT, OutputT> via( final SimpleFunction<InputT, OutputT> fn) { return new MapElements<>( Contextful.fn(fn), fn, fn.getInputTypeDescriptor(), fn.getOutputTypeDescriptor()); }