From fcdc406583b74ccc52a61bc8f8924f8076bfcf8c Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 9 Aug 2016 17:16:00 -0700 Subject: [PATCH 1/4] Rewrites DoFnReflector to go via DoFnSignature DoFnSignature encapsulates type information about a DoFn, in particular which arguments/features its methods actually use. Before this commit, DoFnReflector would parse/verify/generate code in one go; after this commit, these stages are separated: DoFnSignature encapsulates all information needed to generate the code. Additionally, removes the unnecessary genericity in the implementation of DoFnReflector's code generation for the very different methods processElement and start/finishBundle. The code is simpler if decomposed into utility functions, rather than attempting a uniform representation for different methods. --- .../beam/sdk/transforms/DoFnAdapters.java | 271 ++++ .../beam/sdk/transforms/DoFnInvoker.java | 59 + .../beam/sdk/transforms/DoFnInvokers.java | 500 +++++++ .../beam/sdk/transforms/DoFnReflector.java | 1260 +++-------------- .../beam/sdk/transforms/DoFnSignature.java | 148 ++ .../beam/sdk/transforms/DoFnTester.java | 2 +- .../org/apache/beam/sdk/transforms/ParDo.java | 6 +- .../beam/sdk/transforms/DoFnInvokersTest.java | 474 +++++++ .../sdk/transforms/DoFnReflectorTest.java | 881 +++--------- .../beam/sdk/transforms/FlattenTest.java | 4 +- ...elper.java => DoFnInvokersTestHelper.java} | 7 +- .../transforms/DoFnReflectorBenchmark.java | 9 +- 12 files changed, 1898 insertions(+), 1723 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java rename sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/{DoFnReflectorTestHelper.java => DoFnInvokersTestHelper.java} (93%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java new file mode 100644 index 000000000000..6079c184036b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -0,0 +1,271 @@ +/* + * 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.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptor; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; + +/** Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. */ +public class DoFnAdapters { + /** Should not be instantiated. */ + private DoFnAdapters() {} + + /** + * If this is an {@link OldDoFn} produced via {@link #toOldDoFn}, returns the class of the + * original {@link DoFn}, otherwise returns {@code fn.getClass()}. + */ + public static Class getDoFnClass(OldDoFn fn) { + if (fn instanceof SimpleDoFnAdapter) { + return ((SimpleDoFnAdapter) fn).fn.getClass(); + } else { + return fn.getClass(); + } + } + + /** Create a {@link OldDoFn} that the {@link DoFn}. */ + public static OldDoFn toOldDoFn(DoFn fn) { + DoFnSignature signature = DoFnReflector.getOrParseSignature(fn.getClass()); + if (signature.getProcessElement().usesSingleWindow()) { + return new WindowDoFnAdapter<>(fn); + } else { + return new SimpleDoFnAdapter<>(fn); + } + } + + /** + * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link + * OldDoFn}. + */ + private static class SimpleDoFnAdapter extends OldDoFn { + private final DoFn fn; + private transient DoFnInvoker invoker; + + SimpleDoFnAdapter(DoFn fn) { + super(fn.aggregators); + this.fn = fn; + this.invoker = DoFnReflector.newByteBuddyInvoker(fn); + } + + @Override + public void setup() throws Exception { + this.invoker.invokeSetup(); + } + + @Override + public void startBundle(Context c) throws Exception { + this.fn.prepareForProcessing(); + invoker.invokeStartBundle(new ContextAdapter<>(fn, c)); + } + + @Override + public void finishBundle(Context c) throws Exception { + invoker.invokeFinishBundle(new ContextAdapter<>(fn, c)); + } + + @Override + public void teardown() throws Exception { + this.invoker.invokeTeardown(); + } + + @Override + public void processElement(ProcessContext c) throws Exception { + ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); + invoker.invokeProcessElement(adapter, adapter); + } + + @Override + protected TypeDescriptor getInputTypeDescriptor() { + return fn.getInputTypeDescriptor(); + } + + @Override + protected TypeDescriptor getOutputTypeDescriptor() { + return fn.getOutputTypeDescriptor(); + } + + @Override + public Duration getAllowedTimestampSkew() { + return fn.getAllowedTimestampSkew(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.include(fn); + } + + private void readObject(java.io.ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.invoker = DoFnReflector.newByteBuddyInvoker(fn); + } + } + + /** Wraps a {@link DoFn} that requires access to {@link BoundedWindow} as an {@link OldDoFn}. */ + private static class WindowDoFnAdapter extends SimpleDoFnAdapter + implements OldDoFn.RequiresWindowAccess { + + WindowDoFnAdapter(DoFn fn) { + super(fn); + } + } + + /** + * Wraps an {@link OldDoFn.Context} as a {@link DoFn.ExtraContextFactory} inside a {@link + * DoFn.StartBundle} or {@link DoFn.FinishBundle} method, which means the extra context is + * unavailable. + */ + private static class ContextAdapter extends DoFn.Context + implements DoFn.ExtraContextFactory { + + private OldDoFn.Context context; + + private ContextAdapter(DoFn fn, OldDoFn.Context context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + context.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public BoundedWindow window() { + // The DoFn doesn't allow us to ask for these outside ProcessElements, so this + // should be unreachable. + throw new UnsupportedOperationException("Can only get the window in ProcessElements"); + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException("inputProvider() exists only for testing"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("outputReceiver() exists only for testing"); + } + } + + /** + * Wraps an {@link OldDoFn.ProcessContext} as a {@link DoFn.ExtraContextFactory} inside a {@link + * DoFn.ProcessElement} method. + */ + private static class ProcessContextAdapter + extends DoFn.ProcessContext + implements DoFn.ExtraContextFactory { + + private OldDoFn.ProcessContext context; + + private ProcessContextAdapter( + DoFn fn, OldDoFn.ProcessContext context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return context.sideInput(view); + } + + @Override + public void output(OutputT output) { + context.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public InputT element() { + return context.element(); + } + + @Override + public Instant timestamp() { + return context.timestamp(); + } + + @Override + public PaneInfo pane() { + return context.pane(); + } + + @Override + public BoundedWindow window() { + return context.window(); + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException("inputProvider() exists only for testing"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("outputReceiver() exists only for testing"); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java new file mode 100644 index 000000000000..6785263fb352 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java @@ -0,0 +1,59 @@ +/* + * 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; + +/** + * Interface for invoking the {@code DoFn} processing methods. + * + * Instantiating a {@link DoFnInvoker} associates it with a specific {@link DoFn} instance, + * referred to as the bound {@link DoFn}. + */ +public interface DoFnInvoker { + /** + * Invoke the {@link DoFn.Setup} method on the bound {@link DoFn}. + */ + void invokeSetup(); + + /** + * Invoke the {@link DoFn.StartBundle} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.Context} to invoke the fn with. + */ + void invokeStartBundle(DoFn.Context c); + + /** + * Invoke the {@link DoFn.FinishBundle} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.Context} to invoke the fn with. + */ + void invokeFinishBundle(DoFn.Context c); + + /** + * Invoke the {@link DoFn.Teardown} method on the bound {@link DoFn}. + */ + void invokeTeardown(); + + /** + * Invoke the {@link DoFn.ProcessElement} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.ProcessContext} to invoke the fn with. + * @param extra Factory for producing extra parameter objects (such as window), if necessary. + */ + void invokeProcessElement( + DoFn.ProcessContext c, DoFn.ExtraContextFactory extra); +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java new file mode 100644 index 000000000000..93687856bf78 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java @@ -0,0 +1,500 @@ +/* + * 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 org.apache.beam.sdk.transforms.DoFn.FinishBundle; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; +import org.apache.beam.sdk.util.UserCodeException; + +import com.google.common.base.Preconditions; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.NamingStrategy; +import net.bytebuddy.description.field.FieldDescription; +import net.bytebuddy.description.method.MethodDescription; +import net.bytebuddy.description.modifier.FieldManifestation; +import net.bytebuddy.description.modifier.Visibility; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.DynamicType; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.dynamic.scaffold.InstrumentedType; +import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy; +import net.bytebuddy.implementation.Implementation; +import net.bytebuddy.implementation.MethodCall; +import net.bytebuddy.implementation.bind.MethodDelegationBinder; +import net.bytebuddy.implementation.bytecode.ByteCodeAppender; +import net.bytebuddy.implementation.bytecode.StackManipulation; +import net.bytebuddy.implementation.bytecode.Throw; +import net.bytebuddy.implementation.bytecode.member.FieldAccess; +import net.bytebuddy.implementation.bytecode.member.MethodInvocation; +import net.bytebuddy.implementation.bytecode.member.MethodReturn; +import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; +import net.bytebuddy.jar.asm.Label; +import net.bytebuddy.jar.asm.MethodVisitor; +import net.bytebuddy.jar.asm.Opcodes; +import net.bytebuddy.matcher.ElementMatchers; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumMap; +import java.util.LinkedHashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +/** Dynamically generates {@link DoFnInvoker} instances for invoking a {@link DoFn}. */ +abstract class DoFnInvokers { + /** + * A cache of constructors of generated {@link DoFnInvoker} classes, keyed by {@link DoFn} class. + * Needed because generating an invoker class is expensive, and to avoid generating an excessive + * number of classes consuming PermGen memory. + */ + private static final Map, Constructor> BYTE_BUDDY_INVOKER_CONSTRUCTOR_CACHE = + new LinkedHashMap<>(); + + private static final String FN_DELEGATE_FIELD_NAME = "delegate"; + + /** This is a factory class that should not be instantiated. */ + private DoFnInvokers() {} + + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + static DoFnInvoker newByteBuddyInvoker( + DoFnSignature signature, DoFn fn) { + Preconditions.checkArgument( + signature.getFnClass().equals(fn.getClass()), + "Signature is for class %s, but fn is of class %s", + signature.getFnClass(), + fn.getClass()); + try { + @SuppressWarnings("unchecked") + DoFnInvoker invoker = + (DoFnInvoker) + getOrGenerateByteBuddyInvokerConstructor(signature).newInstance(fn); + return invoker; + } catch (InstantiationException + | IllegalAccessException + | IllegalArgumentException + | InvocationTargetException + | SecurityException e) { + throw new RuntimeException("Unable to bind invoker for " + fn.getClass(), e); + } + } + + /** + * Returns a generated constructor for a {@link DoFnInvoker} for the given {@link DoFn} class and + * caches it. + */ + private static synchronized Constructor getOrGenerateByteBuddyInvokerConstructor( + DoFnSignature signature) { + Class fnClass = signature.getFnClass(); + Constructor constructor = BYTE_BUDDY_INVOKER_CONSTRUCTOR_CACHE.get(fnClass); + if (constructor != null) { + return constructor; + } + Class> invokerClass = generateInvokerClass(signature); + try { + constructor = invokerClass.getConstructor(fnClass); + } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) { + throw new RuntimeException(e); + } + BYTE_BUDDY_INVOKER_CONSTRUCTOR_CACHE.put(fnClass, constructor); + return constructor; + } + + /** Generates a {@link DoFnInvoker} class for the given {@link DoFnSignature}. */ + private static Class> generateInvokerClass(DoFnSignature signature) { + Class fnClass = signature.getFnClass(); + + final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(fnClass); + + DynamicType.Builder builder = + new ByteBuddy() + // Create subclasses inside the target class, to have access to + // private and package-private bits + .with( + new NamingStrategy.SuffixingRandom("auxiliary") { + @Override + public String subclass(TypeDescription.Generic superClass) { + return super.name(clazzDescription); + } + }) + // Create a subclass of DoFnInvoker + .subclass(DoFnInvoker.class, ConstructorStrategy.Default.NO_CONSTRUCTORS) + .defineField( + FN_DELEGATE_FIELD_NAME, fnClass, Visibility.PRIVATE, FieldManifestation.FINAL) + .defineConstructor(Visibility.PUBLIC) + .withParameter(fnClass) + .intercept(new InvokerConstructor()) + // Delegate processElement(), startBundle() and finishBundle() to the fn. + .method(ElementMatchers.named("invokeProcessElement")) + .intercept(new ProcessElementDelegation(signature.getProcessElement())) + .method(ElementMatchers.named("invokeStartBundle")) + .intercept( + signature.getStartBundle() == null + ? new NoopMethodImplementation() + : new BundleMethodDelegation(signature.getStartBundle())) + .method(ElementMatchers.named("invokeFinishBundle")) + .intercept( + signature.getFinishBundle() == null + ? new NoopMethodImplementation() + : new BundleMethodDelegation(signature.getFinishBundle())) + .method(ElementMatchers.named("invokeSetup")) + .intercept( + signature.getSetup() == null + ? new NoopMethodImplementation() + : new LifecycleMethodDelegation(signature.getSetup())) + .method(ElementMatchers.named("invokeTeardown")) + .intercept( + signature.getTeardown() == null + ? new NoopMethodImplementation() + : new LifecycleMethodDelegation(signature.getTeardown())); + + DynamicType.Unloaded unloaded = builder.make(); + + @SuppressWarnings("unchecked") + Class> res = + (Class>) + unloaded + .load(DoFnInvokers.class.getClassLoader(), ClassLoadingStrategy.Default.INJECTION) + .getLoaded(); + return res; + } + + /** Implements an invoker method by doing nothing and immediately returning void. */ + private static class NoopMethodImplementation implements Implementation { + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation manipulation = MethodReturn.VOID; + StackManipulation.Size size = manipulation.apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + } + + /** + * Base class for implementing an invoker method by delegating to a method of the target {@link + * DoFn}. + */ + private abstract static class MethodDelegation implements Implementation { + FieldDescription delegateField; + + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + // Remember the field description of the instrumented type. + delegateField = + instrumentedType + .getDeclaredFields() + .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) + .getOnly(); + + // Delegating the method call doesn't require any changes to the instrumented type. + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation manipulation = + new StackManipulation.Compound( + // Push "this" reference to the stack + MethodVariableAccess.REFERENCE.loadOffset(0), + // Access the delegate field of the the invoker + FieldAccess.forField(delegateField).getter(), + invokeTargetMethod(instrumentedMethod)); + StackManipulation.Size size = manipulation.apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + + /** + * Generates code to invoke the target method. When this is called the delegate field will be on + * top of the stack. This should add any necessary arguments to the stack and then perform the + * method invocation. + */ + protected abstract StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod); + } + + /** + * Implements the invoker's {@link DoFnInvoker#invokeProcessElement} method by delegating to the + * {@link DoFn.ProcessElement} method. + */ + private static final class ProcessElementDelegation extends MethodDelegation { + private static final Map + EXTRA_CONTEXT_FACTORY_METHODS; + + static { + try { + Map methods = + new EnumMap<>(DoFnSignature.ProcessElementMethod.Parameter.class); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("window"))); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("inputProvider"))); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("outputReceiver"))); + EXTRA_CONTEXT_FACTORY_METHODS = Collections.unmodifiableMap(methods); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private final DoFnSignature.ProcessElementMethod signature; + + /** Implementation of {@link MethodDelegation} for the {@link ProcessElement} method. */ + private ProcessElementDelegation(DoFnSignature.ProcessElementMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(signature.getTargetMethod())) + .resolve(instrumentedMethod); + + // Parameters of the wrapper invoker method: + // DoFn.ProcessContext, ExtraContextFactory. + // Parameters of the wrapped DoFn method: + // DoFn.ProcessContext, [BoundedWindow, InputProvider, OutputReceiver] in any order + ArrayList parameters = new ArrayList<>(); + // Push the ProcessContext argument. + parameters.add(MethodVariableAccess.REFERENCE.loadOffset(1)); + // Push the extra arguments in their actual order. + StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadOffset(2); + for (DoFnSignature.ProcessElementMethod.Parameter param : signature.getExtraParameters()) { + parameters.add( + new StackManipulation.Compound( + pushExtraContextFactory, + MethodInvocation.invoke(EXTRA_CONTEXT_FACTORY_METHODS.get(param)))); + } + + return new StackManipulation.Compound( + // Push the parameters + new StackManipulation.Compound(parameters), + // Invoke the target method + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + // Return from the instrumented method + MethodReturn.VOID); + } + } + + /** + * Implements {@link DoFnInvoker#invokeStartBundle} or {@link DoFnInvoker#invokeFinishBundle} by + * delegating respectively to the {@link StartBundle} and {@link FinishBundle} methods. + */ + private static final class BundleMethodDelegation extends MethodDelegation { + private final DoFnSignature.BundleMethod signature; + + private BundleMethodDelegation(@Nullable DoFnSignature.BundleMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(checkNotNull(signature).getTargetMethod())) + .resolve(instrumentedMethod); + return new StackManipulation.Compound( + // Push the parameters + MethodVariableAccess.REFERENCE.loadOffset(1), + // Invoke the target method + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + MethodReturn.VOID); + } + } + + /** + * Implements {@link DoFnInvoker#invokeSetup} or {@link DoFnInvoker#invokeTeardown} by + * delegating respectively to the {@link Setup} and {@link Teardown} methods. + */ + private static final class LifecycleMethodDelegation extends MethodDelegation { + private final DoFnSignature.LifecycleMethod signature; + + private LifecycleMethodDelegation(@Nullable DoFnSignature.LifecycleMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(checkNotNull(signature).getTargetMethod())) + .resolve(instrumentedMethod); + return new StackManipulation.Compound( + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + MethodReturn.VOID); + } + } + + /** + * Wraps a given stack manipulation in a try catch block. Any exceptions thrown within the try are + * wrapped with a {@link UserCodeException}. + */ + private static StackManipulation wrapWithUserCodeException(final StackManipulation tryBody) { + final MethodDescription createUserCodeException; + try { + createUserCodeException = + new MethodDescription.ForLoadedMethod( + UserCodeException.class.getDeclaredMethod("wrap", Throwable.class)); + } catch (NoSuchMethodException | SecurityException e) { + throw new RuntimeException("Unable to find UserCodeException.wrap", e); + } + + return new StackManipulation() { + @Override + public boolean isValid() { + return tryBody.isValid(); + } + + @Override + public Size apply(MethodVisitor mv, Implementation.Context implementationContext) { + Label tryBlockStart = new Label(); + Label tryBlockEnd = new Label(); + Label catchBlockStart = new Label(); + Label catchBlockEnd = new Label(); + + String throwableName = new TypeDescription.ForLoadedType(Throwable.class).getInternalName(); + mv.visitTryCatchBlock(tryBlockStart, tryBlockEnd, catchBlockStart, throwableName); + + // The try block attempts to perform the expected operations, then jumps to success + mv.visitLabel(tryBlockStart); + Size trySize = tryBody.apply(mv, implementationContext); + mv.visitJumpInsn(Opcodes.GOTO, catchBlockEnd); + mv.visitLabel(tryBlockEnd); + + // The handler wraps the exception, and then throws. + mv.visitLabel(catchBlockStart); + // Add the exception to the frame + mv.visitFrame( + Opcodes.F_SAME1, + // No local variables + 0, + new Object[] {}, + // 1 stack element (the throwable) + 1, + new Object[] {throwableName}); + + Size catchSize = + new Compound(MethodInvocation.invoke(createUserCodeException), Throw.INSTANCE) + .apply(mv, implementationContext); + + mv.visitLabel(catchBlockEnd); + // The frame contents after the try/catch block is the same + // as it was before. + mv.visitFrame( + Opcodes.F_SAME, + // No local variables + 0, + new Object[] {}, + // No new stack variables + 0, + new Object[] {}); + + return new Size( + trySize.getSizeImpact(), + Math.max(trySize.getMaximalSize(), catchSize.getMaximalSize())); + } + }; + } + + /** + * A constructor {@link Implementation} for a {@link DoFnInvoker class}. Produces the byte code + * for a constructor that takes a single argument and assigns it to the delegate field. + */ + private static final class InvokerConstructor implements Implementation { + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation.Size size = + new StackManipulation.Compound( + // Load the this reference + MethodVariableAccess.REFERENCE.loadOffset(0), + // Invoke the super constructor (default constructor of Object) + MethodInvocation.invoke( + new TypeDescription.ForLoadedType(Object.class) + .getDeclaredMethods() + .filter( + ElementMatchers.isConstructor() + .and(ElementMatchers.takesArguments(0))) + .getOnly()), + // Load the this reference + MethodVariableAccess.REFERENCE.loadOffset(0), + // Load the delegate argument + MethodVariableAccess.REFERENCE.loadOffset(1), + // Assign the delegate argument to the delegate field + FieldAccess.forField( + implementationTarget + .getInstrumentedType() + .getDeclaredFields() + .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) + .getOnly()) + .putter(), + // Return void. + MethodReturn.VOID) + .apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java index bf040412b5e0..20bf97ee8500 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java @@ -17,72 +17,16 @@ */ package org.apache.beam.sdk.transforms; -import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.StartBundle; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; -import net.bytebuddy.ByteBuddy; -import net.bytebuddy.NamingStrategy.SuffixingRandom; -import net.bytebuddy.description.field.FieldDescription; -import net.bytebuddy.description.method.MethodDescription; -import net.bytebuddy.description.method.ParameterList; -import net.bytebuddy.description.modifier.FieldManifestation; -import net.bytebuddy.description.modifier.Visibility; -import net.bytebuddy.description.type.TypeDescription; -import net.bytebuddy.description.type.TypeDescription.Generic; -import net.bytebuddy.dynamic.DynamicType; -import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; -import net.bytebuddy.dynamic.scaffold.InstrumentedType; -import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy.Default; -import net.bytebuddy.implementation.Implementation; -import net.bytebuddy.implementation.MethodCall.MethodLocator; -import net.bytebuddy.implementation.StubMethod; -import net.bytebuddy.implementation.bind.MethodDelegationBinder.MethodInvoker; -import net.bytebuddy.implementation.bind.annotation.TargetMethodAnnotationDrivenBinder.TerminationHandler; -import net.bytebuddy.implementation.bytecode.ByteCodeAppender; -import net.bytebuddy.implementation.bytecode.Duplication; -import net.bytebuddy.implementation.bytecode.StackManipulation; -import net.bytebuddy.implementation.bytecode.Throw; -import net.bytebuddy.implementation.bytecode.assign.Assigner; -import net.bytebuddy.implementation.bytecode.member.FieldAccess; -import net.bytebuddy.implementation.bytecode.member.MethodInvocation; -import net.bytebuddy.implementation.bytecode.member.MethodReturn; -import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; -import net.bytebuddy.jar.asm.Label; -import net.bytebuddy.jar.asm.MethodVisitor; -import net.bytebuddy.jar.asm.Opcodes; -import net.bytebuddy.matcher.ElementMatchers; -import org.joda.time.Duration; -import org.joda.time.Instant; - -import java.io.IOException; import java.lang.annotation.Annotation; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.lang.reflect.ParameterizedType; @@ -95,1056 +39,288 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - /** - * Utility implementing the necessary reflection for working with {@link DoFn}s. + * Parses a {@link DoFn} and computes its {@link DoFnSignature}. See {@link #getOrParseSignature}. */ public abstract class DoFnReflector { + private DoFnReflector() {} - private static final String FN_DELEGATE_FIELD_NAME = "delegate"; - - private enum Availability { - /** Indicates parameters only available in {@code @ProcessElement} methods. */ - PROCESS_ELEMENT_ONLY, - /** Indicates parameters available in all methods. */ - EVERYWHERE - } - - /** - * Enumeration of the parameters available from the {@link ExtraContextFactory} to use as - * additional parameters for {@link DoFn} methods. - *

- * We don't rely on looking for properly annotated methods within {@link ExtraContextFactory} - * because erasure would make it impossible to completely fill in the type token for context - * parameters that depend on the input/output type. - */ - private enum AdditionalParameter { - - /** Any {@link BoundedWindow} parameter is populated by the window of the current element. */ - WINDOW_OF_ELEMENT(Availability.PROCESS_ELEMENT_ONLY, BoundedWindow.class, "window") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return TypeToken.of(BoundedWindow.class); - } - }, - - INPUT_PROVIDER(Availability.PROCESS_ELEMENT_ONLY, DoFn.InputProvider.class, "inputProvider") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return new TypeToken>() {}.where( - new TypeParameter() {}, in); - } - - @Override - public boolean isHidden() { - return true; - } - }, - - OUTPUT_RECEIVER( - Availability.PROCESS_ELEMENT_ONLY, DoFn.OutputReceiver.class, "outputReceiver") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return new TypeToken>() {}.where( - new TypeParameter() {}, out); - } + private static final Map, DoFnSignature> SIGNATURE_CACHE = new LinkedHashMap<>(); - @Override - public boolean isHidden() { - return true; - } - }; - - /** - * Create a type token representing the given parameter. May use the type token associated - * with the input and output types of the {@link DoFn}, depending on the extra - * context. - */ - abstract TypeToken tokenFor( - TypeToken in, TypeToken out); - - /** - * Indicates whether this enum is for testing only, hence should not appear in error messages, - * etc. Defaults to {@code false}. - */ - boolean isHidden() { - return false; + /** @return the {@link DoFnSignature} for the given {@link DoFn}. */ + static synchronized DoFnSignature getOrParseSignature( + @SuppressWarnings("rawtypes") Class fn) { + DoFnSignature signature = SIGNATURE_CACHE.get(fn); + if (signature != null) { + return signature; } - private final Class rawType; - private final Availability availability; - private final transient MethodDescription method; - - private AdditionalParameter(Availability availability, Class rawType, String method) { - this.availability = availability; - this.rawType = rawType; - try { - this.method = new MethodDescription.ForLoadedMethod( - ExtraContextFactory.class.getMethod(method)); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException( - "Unable to access method " + method + " on " + ExtraContextFactory.class, e); - } - } + signature = parseSignature(fn); + SIGNATURE_CACHE.put(fn, signature); + return signature; } - private static final Map, AdditionalParameter> EXTRA_CONTEXTS; - private static final Map, AdditionalParameter> EXTRA_PROCESS_CONTEXTS; - - static { - ImmutableMap.Builder, AdditionalParameter> everywhereBuilder = - ImmutableMap., AdditionalParameter>builder(); - ImmutableMap.Builder, AdditionalParameter> processElementBuilder = - ImmutableMap., AdditionalParameter>builder(); + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + public static DoFnInvoker newByteBuddyInvoker( + DoFn fn) { + return DoFnInvokers.newByteBuddyInvoker(getOrParseSignature(fn.getClass()), fn); + } - for (AdditionalParameter value : AdditionalParameter.values()) { - switch (value.availability) { - case EVERYWHERE: - everywhereBuilder.put(value.rawType, value); - break; - case PROCESS_ELEMENT_ONLY: - processElementBuilder.put(value.rawType, value); - break; + /** Analyzes a given {@link DoFn} class and extracts its {@link DoFnSignature}. */ + private static DoFnSignature parseSignature(Class fnClass) { + TypeToken inputT = null; + TypeToken outputT = null; + + // Extract the input and output type. + checkArgument( + DoFn.class.isAssignableFrom(fnClass), + "%s must be subtype of DoFn", + fnClass.getSimpleName()); + TypeToken fnToken = TypeToken.of(fnClass); + for (TypeToken supertype : fnToken.getTypes()) { + if (!supertype.getRawType().equals(DoFn.class)) { + continue; } - } - - EXTRA_CONTEXTS = everywhereBuilder.build(); - EXTRA_PROCESS_CONTEXTS = processElementBuilder - // Process Element contexts include everything available everywhere - .putAll(EXTRA_CONTEXTS) - .build(); + Type[] args = ((ParameterizedType) supertype.getType()).getActualTypeArguments(); + inputT = TypeToken.of(args[0]); + outputT = TypeToken.of(args[1]); + } + Preconditions.checkNotNull(inputT, "Unable to determine input type from %s", fnClass); + + Method processElementMethod = findAnnotatedMethod(DoFn.ProcessElement.class, fnClass, true); + Method startBundleMethod = findAnnotatedMethod(DoFn.StartBundle.class, fnClass, false); + Method finishBundleMethod = findAnnotatedMethod(DoFn.FinishBundle.class, fnClass, false); + Method setupMethod = findAnnotatedMethod(DoFn.Setup.class, fnClass, false); + Method teardownMethod = findAnnotatedMethod(DoFn.Teardown.class, fnClass, false); + + return new DoFnSignature( + fnClass, + inputT, + outputT, + analyzeProcessElementMethod(fnToken, processElementMethod, inputT, outputT), + (startBundleMethod == null) + ? null + : analyzeBundleMethod(fnToken, startBundleMethod, inputT, outputT), + (finishBundleMethod == null) + ? null + : analyzeBundleMethod(fnToken, finishBundleMethod, inputT, outputT), + (setupMethod == null) ? null : analyzeLifecycleMethod(setupMethod), + (teardownMethod == null) ? null : analyzeLifecycleMethod(teardownMethod)); } /** - * @return true if the reflected {@link DoFn} uses a Single Window. - */ - public abstract boolean usesSingleWindow(); - - /** Create an {@link DoFnInvoker} bound to the given {@link OldDoFn}. */ - public abstract DoFnInvoker bindInvoker( - DoFn fn); - - private static final Map, DoFnReflector> REFLECTOR_CACHE = - new LinkedHashMap, DoFnReflector>(); - - /** - * @return the {@link DoFnReflector} for the given {@link DoFn}. + * Generates a type token for {@code DoFn.ProcessContext} given {@code InputT} + * and {@code OutputT}. */ - public static DoFnReflector of( - @SuppressWarnings("rawtypes") Class fn) { - DoFnReflector reflector = REFLECTOR_CACHE.get(fn); - if (reflector != null) { - return reflector; - } - - reflector = new GenericDoFnReflector(fn); - REFLECTOR_CACHE.put(fn, reflector); - return reflector; + private static + TypeToken.ProcessContext> doFnProcessContextTypeOf( + TypeToken inputT, TypeToken outputT) { + return new TypeToken.ProcessContext>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); } /** - * Create a {@link OldDoFn} that the {@link DoFn}. + * Generates a type token for {@code DoFn.Context} given {@code InputT} and + * {@code OutputT}. */ - public OldDoFn toDoFn(DoFn fn) { - if (usesSingleWindow()) { - return new WindowDoFnAdapter(this, fn); - } else { - return new SimpleDoFnAdapter(this, fn); - } + private static TypeToken.Context> doFnContextTypeOf( + TypeToken inputT, TypeToken outputT) { + return new TypeToken.Context>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); } - private static String formatType(TypeToken t) { - return ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(t.getType()); + /** Generates a type token for {@code DoFn.InputProvider} given {@code InputT}. */ + private static TypeToken> inputProviderTypeOf( + TypeToken inputT) { + return new TypeToken>() {}.where( + new TypeParameter() {}, inputT); } - private static String format(Method m) { - return ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(m); + /** Generates a type token for {@code DoFn.OutputReceiver} given {@code OutputT}. */ + private static TypeToken> outputReceiverTypeOf( + TypeToken inputT) { + return new TypeToken>() {}.where( + new TypeParameter() {}, inputT); } - private static Collection describeSupportedTypes( - Map, AdditionalParameter> extraProcessContexts, - final TypeToken in, final TypeToken out) { - return FluentIterable - .from(extraProcessContexts.values()) - .filter(new Predicate() { - @Override - public boolean apply(@Nonnull AdditionalParameter additionalParameter) { - return !additionalParameter.isHidden(); - } - }) - .transform(new Function() { - @Override - @Nonnull - public String apply(@Nonnull AdditionalParameter input) { - return formatType(input.tokenFor(in, out)); - } - }) - .toSortedSet(String.CASE_INSENSITIVE_ORDER); - } + static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( + TypeToken fnClass, Method m, TypeToken inputT, TypeToken outputT) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument(!m.isVarArgs(), "%s must not have var args", format(m)); - @VisibleForTesting - static List verifyProcessMethodArguments(Method m) { - return verifyMethodArguments(m, - EXTRA_PROCESS_CONTEXTS, - new TypeToken.ProcessContext>() {}, - new TypeParameter() {}, - new TypeParameter() {}); - } + TypeToken processContextToken = doFnProcessContextTypeOf(inputT, outputT); - @VisibleForTesting - static List verifyBundleMethodArguments(Method m) { - if (m == null) { - return null; - } - return verifyMethodArguments(m, - EXTRA_CONTEXTS, - new TypeToken.Context>() {}, - new TypeParameter() {}, - new TypeParameter() {}); - } - - @VisibleForTesting - static void verifyLifecycleMethodArguments(Method m) { - if (m == null) { - return; - } - checkState(void.class.equals(m.getReturnType()), "%s must have void return type", format(m)); - checkState(m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); - } - - /** - * Verify the method arguments for a given {@link DoFn} method. - * - *

The requirements for a method to be valid, are: - *

    - *
  1. The method has at least one argument. - *
  2. The first argument is of type firstContextArg. - *
  3. The remaining arguments have raw types that appear in {@code contexts} - *
  4. Any generics on the extra context arguments match what is expected. Currently, this - * is exercised only by placeholders. For example, {@code InputReceiver must either match - * the {@code InputT} {@code OldDoFn.ProcessContext} or use a wildcard, etc. - *
- * - * @param m the method to verify - * @param contexts mapping from raw classes to the {@link AdditionalParameter} used - * to create new instances. - * @param firstContextArg the expected type of the first context argument - * @param iParam TypeParameter representing the input type - * @param oParam TypeParameter representing the output type - */ - @VisibleForTesting - static List verifyMethodArguments( - Method m, - Map, AdditionalParameter> contexts, - TypeToken firstContextArg, - TypeParameter iParam, - TypeParameter oParam) { - - if (!void.class.equals(m.getReturnType())) { - throw new IllegalStateException(String.format( - "%s must have a void return type", format(m))); - } - if (m.isVarArgs()) { - throw new IllegalStateException(String.format( - "%s must not have var args", format(m))); - } - - // The first parameter must be present, and must be the specified type Type[] params = m.getGenericParameterTypes(); TypeToken contextToken = null; if (params.length > 0) { - contextToken = TypeToken.of(params[0]); - } - if (contextToken == null - || !contextToken.getRawType().equals(firstContextArg.getRawType())) { - throw new IllegalStateException(String.format( - "%s must take a %s as its first argument", - format(m), firstContextArg.getRawType().getSimpleName())); - } - AdditionalParameter[] contextInfos = new AdditionalParameter[params.length - 1]; - - // Fill in the generics in the allExtraContextArgs interface from the types in the - // Context or ProcessContext OldDoFn. - ParameterizedType pt = (ParameterizedType) contextToken.getType(); - // We actually want the owner, since ProcessContext and Context are owned by DoFn. - pt = (ParameterizedType) pt.getOwnerType(); - @SuppressWarnings("unchecked") - TypeToken iActual = (TypeToken) TypeToken.of(pt.getActualTypeArguments()[0]); - @SuppressWarnings("unchecked") - TypeToken oActual = (TypeToken) TypeToken.of(pt.getActualTypeArguments()[1]); - - // All of the remaining parameters must be a super-interface of allExtraContextArgs - // that is not listed in the EXCLUDED_INTERFACES set. - for (int i = 1; i < params.length; i++) { - TypeToken param = TypeToken.of(params[i]); - - AdditionalParameter info = contexts.get(param.getRawType()); - if (info == null) { - throw new IllegalStateException(String.format( + contextToken = fnClass.resolveType(params[0]); + } + checkArgument( + contextToken != null && contextToken.equals(processContextToken), + "%s must take a %s as its first argument", + format(m), + formatType(processContextToken)); + + List extraParameters = new ArrayList<>(); + TypeToken expectedInputProviderT = inputProviderTypeOf(inputT); + TypeToken expectedOutputReceiverT = outputReceiverTypeOf(outputT); + for (int i = 1; i < params.length; ++i) { + TypeToken param = fnClass.resolveType(params[i]); + Class rawType = param.getRawType(); + if (rawType.equals(BoundedWindow.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW), + "Multiple BoundedWindow parameters in %s", + format(m)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW); + } else if (rawType.equals(DoFn.InputProvider.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER), + "Multiple InputProvider parameters in %s", + format(m)); + checkArgument( + param.equals(expectedInputProviderT), + "Wrong type of InputProvider parameter for method %s: %s, should be %s", + format(m), + formatType(param), + formatType(expectedInputProviderT)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER); + } else if (rawType.equals(DoFn.OutputReceiver.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER), + "Multiple OutputReceiver parameters in %s", + format(m)); + checkArgument( + param.equals(expectedOutputReceiverT), + "Wrong type of OutputReceiver parameter for method %s: %s, should be %s", + format(m), + formatType(param), + formatType(expectedOutputReceiverT)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER); + } else { + List allowedParamTypes = + Arrays.asList(formatType(new TypeToken() {})); + checkArgument( + false, "%s is not a valid context parameter for method %s. Should be one of %s", - formatType(param), format(m), - describeSupportedTypes(contexts, iActual, oActual))); - } - - // If we get here, the class matches, but maybe the generics don't: - TypeToken expected = info.tokenFor(iActual, oActual); - if (!expected.isSubtypeOf(param)) { - throw new IllegalStateException(String.format( - "Incompatible generics in context parameter %s for method %s. Should be %s", - formatType(param), format(m), formatType(info.tokenFor(iActual, oActual)))); + formatType(param), + format(m), + allowedParamTypes); } - - // Register the (now validated) context info - contextInfos[i - 1] = info; } - return ImmutableList.copyOf(contextInfos); - } - - /** Interface for invoking the {@code OldDoFn} processing methods. */ - public interface DoFnInvoker { - /** Invoke {@link OldDoFn#setup} on the bound {@code OldDoFn}. */ - void invokeSetup(); - /** Invoke {@link OldDoFn#startBundle} on the bound {@code OldDoFn}. */ - void invokeStartBundle( - DoFn.Context c, - ExtraContextFactory extra); - /** Invoke {@link OldDoFn#finishBundle} on the bound {@code OldDoFn}. */ - void invokeFinishBundle( - DoFn.Context c, - ExtraContextFactory extra); - - /** Invoke {@link OldDoFn#teardown()} on the bound {@code DoFn}. */ - void invokeTeardown(); - /** Invoke {@link OldDoFn#processElement} on the bound {@code OldDoFn}. */ - public void invokeProcessElement( - DoFn.ProcessContext c, - ExtraContextFactory extra); + return new DoFnSignature.ProcessElementMethod(m, extraParameters); } - /** - * Implementation of {@link DoFnReflector} for the arbitrary {@link DoFn}. - */ - private static class GenericDoFnReflector extends DoFnReflector { - - private final Method setup; - private final Method startBundle; - private final Method processElement; - private final Method finishBundle; - private final Method teardown; - private final List processElementArgs; - private final List startBundleArgs; - private final List finishBundleArgs; - private final Constructor constructor; - - private GenericDoFnReflector( - @SuppressWarnings("rawtypes") Class fn) { - // Locate the annotated methods - this.processElement = findAnnotatedMethod(ProcessElement.class, fn, true); - this.setup = findAnnotatedMethod(Setup.class, fn, false); - this.startBundle = findAnnotatedMethod(StartBundle.class, fn, false); - this.finishBundle = findAnnotatedMethod(FinishBundle.class, fn, false); - this.teardown = findAnnotatedMethod(Teardown.class, fn, false); - - // Verify that their method arguments satisfy our conditions. - this.processElementArgs = verifyProcessMethodArguments(processElement); - this.startBundleArgs = verifyBundleMethodArguments(startBundle); - this.finishBundleArgs = verifyBundleMethodArguments(finishBundle); - verifyLifecycleMethodArguments(setup); - verifyLifecycleMethodArguments(teardown); - - this.constructor = createInvokerConstructor(fn); - } + static DoFnSignature.BundleMethod analyzeBundleMethod( + TypeToken fnToken, Method m, TypeToken inputT, TypeToken outputT) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument(!m.isVarArgs(), "%s must not have var args", format(m)); - private static Collection declaredMethodsWithAnnotation( - Class anno, - Class startClass, Class stopClass) { - Collection matches = new ArrayList<>(); + TypeToken expectedContextToken = doFnContextTypeOf(inputT, outputT); - Class clazz = startClass; - LinkedHashSet> interfaces = new LinkedHashSet<>(); + Type[] params = m.getGenericParameterTypes(); + checkArgument( + params.length == 1, + "%s must have a single argument of type %s", + format(m), + formatType(expectedContextToken)); + TypeToken contextToken = fnToken.resolveType(params[0]); + checkArgument( + contextToken.equals(expectedContextToken), + "Wrong type of context argument to %s: %s, must be %s", + format(m), + formatType(contextToken), + formatType(expectedContextToken)); + + return new DoFnSignature.BundleMethod(m); + } - // First, find all declared methods on the startClass and parents (up to stopClass) - while (clazz != null && !clazz.equals(stopClass)) { - for (Method method : clazz.getDeclaredMethods()) { - if (method.isAnnotationPresent(anno)) { - matches.add(method); - } - } + static DoFnSignature.LifecycleMethod analyzeLifecycleMethod(Method m) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument( + m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); + return new DoFnSignature.LifecycleMethod(m); + } - Collections.addAll(interfaces, clazz.getInterfaces()); + private static Collection declaredMethodsWithAnnotation( + Class anno, Class startClass, Class stopClass) { + Collection matches = new ArrayList<>(); - clazz = clazz.getSuperclass(); - } + Class clazz = startClass; + LinkedHashSet> interfaces = new LinkedHashSet<>(); - // Now, iterate over all the discovered interfaces - for (Method method : ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) { + // First, find all declared methods on the startClass and parents (up to stopClass) + while (clazz != null && !clazz.equals(stopClass)) { + for (Method method : clazz.getDeclaredMethods()) { if (method.isAnnotationPresent(anno)) { matches.add(method); } } - return matches; - } - - private static Method findAnnotatedMethod( - Class anno, Class fnClazz, boolean required) { - Collection matches = declaredMethodsWithAnnotation( - anno, fnClazz, DoFn.class); - - if (matches.size() == 0) { - if (required) { - throw new IllegalStateException(String.format( - "No method annotated with @%s found in %s", - anno.getSimpleName(), fnClazz.getName())); - } else { - return null; - } - } - - // If we have at least one match, then either it should be the only match - // or it should be an extension of the other matches (which came from parent - // classes). - Method first = matches.iterator().next(); - for (Method other : matches) { - if (!first.getName().equals(other.getName()) - || !Arrays.equals(first.getParameterTypes(), other.getParameterTypes())) { - throw new IllegalStateException(String.format( - "Found multiple methods annotated with @%s. [%s] and [%s]", - anno.getSimpleName(), format(first), format(other))); - } - } - - // We need to be able to call it. We require it is public. - if ((first.getModifiers() & Modifier.PUBLIC) == 0) { - throw new IllegalStateException(format(first) + " must be public"); - } - - // And make sure its not static. - if ((first.getModifiers() & Modifier.STATIC) != 0) { - throw new IllegalStateException(format(first) + " must not be static"); - } - - return first; - } - @Override - public boolean usesSingleWindow() { - return usesContext(AdditionalParameter.WINDOW_OF_ELEMENT); - } + Collections.addAll(interfaces, clazz.getInterfaces()); - private boolean usesContext(AdditionalParameter param) { - return processElementArgs.contains(param) - || (startBundleArgs != null && startBundleArgs.contains(param)) - || (finishBundleArgs != null && finishBundleArgs.contains(param)); + clazz = clazz.getSuperclass(); } - /** - * Use ByteBuddy to generate the code for a {@link DoFnInvoker} that invokes the given - * {@link DoFn}. - * @param clazz - * @return - */ - private Constructor> createInvokerConstructor( - @SuppressWarnings("rawtypes") Class clazz) { - - final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(clazz); - - DynamicType.Builder builder = new ByteBuddy() - // Create subclasses inside the target class, to have access to - // private and package-private bits - .with(new SuffixingRandom("auxiliary") { - @Override - public String subclass(Generic superClass) { - return super.name(clazzDescription); - } - }) - // Create a subclass of DoFnInvoker - .subclass(DoFnInvoker.class, Default.NO_CONSTRUCTORS) - .defineField(FN_DELEGATE_FIELD_NAME, clazz, Visibility.PRIVATE, FieldManifestation.FINAL) - // Define a constructor to populate fields appropriately. - .defineConstructor(Visibility.PUBLIC) - .withParameter(clazz) - .intercept(new InvokerConstructor()) - // Implement the three methods by calling into the appropriate functions on the fn. - .method(ElementMatchers.named("invokeProcessElement")) - .intercept(InvokerDelegation.create( - processElement, BeforeDelegation.NOOP, processElementArgs)) - .method(ElementMatchers.named("invokeStartBundle")) - .intercept(InvokerDelegation.create( - startBundle, BeforeDelegation.INVOKE_PREPARE_FOR_PROCESSING, startBundleArgs)) - .method(ElementMatchers.named("invokeFinishBundle")) - .intercept(InvokerDelegation.create(finishBundle, - BeforeDelegation.NOOP, - finishBundleArgs)) - .method(ElementMatchers.named("invokeSetup")) - .intercept(InvokerDelegation.create(setup, - BeforeDelegation.NOOP, - Collections.emptyList())) - .method(ElementMatchers.named("invokeTeardown")) - .intercept(InvokerDelegation.create(teardown, - BeforeDelegation.NOOP, - Collections.emptyList())); - - @SuppressWarnings("unchecked") - Class> dynamicClass = (Class>) builder - .make() - .load(getClass().getClassLoader(), ClassLoadingStrategy.Default.INJECTION) - .getLoaded(); - try { - return dynamicClass.getConstructor(clazz); - } catch (IllegalArgumentException - | NoSuchMethodException - | SecurityException e) { - throw new RuntimeException(e); - } - } - - @Override - public DoFnInvoker bindInvoker( - DoFn fn) { - try { - @SuppressWarnings("unchecked") - DoFnInvoker invoker = - (DoFnInvoker) constructor.newInstance(fn); - return invoker; - } catch (InstantiationException - | IllegalAccessException - | IllegalArgumentException - | InvocationTargetException - | SecurityException e) { - throw new RuntimeException("Unable to bind invoker for " + fn.getClass(), e); + // Now, iterate over all the discovered interfaces + for (Method method : ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) { + if (method.isAnnotationPresent(anno)) { + matches.add(method); } } + return matches; } - private static class ContextAdapter - extends DoFn.Context - implements DoFn.ExtraContextFactory { - - private OldDoFn.Context context; - - private ContextAdapter( - DoFn fn, OldDoFn.Context context) { - fn.super(); - this.context = context; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - context.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - context.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - public BoundedWindow window() { - // The DoFn doesn't allow us to ask for these outside ProcessElements, so this - // should be unreachable. - throw new UnsupportedOperationException("Can only get the window in ProcessElements"); - } - - @Override - public DoFn.InputProvider inputProvider() { - throw new UnsupportedOperationException("inputProvider() exists only for testing"); - } + private static Method findAnnotatedMethod( + Class anno, Class fnClazz, boolean required) { + Collection matches = declaredMethodsWithAnnotation(anno, fnClazz, DoFn.class); - @Override - public DoFn.OutputReceiver outputReceiver() { - throw new UnsupportedOperationException("outputReceiver() exists only for testing"); - } - } - - private static class ProcessContextAdapter - extends DoFn.ProcessContext - implements DoFn.ExtraContextFactory { - - private OldDoFn.ProcessContext context; - - private ProcessContextAdapter( - DoFn fn, - OldDoFn.ProcessContext context) { - fn.super(); - this.context = context; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - return context.sideInput(view); - } - - @Override - public void output(OutputT output) { - context.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - context.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - public InputT element() { - return context.element(); - } - - @Override - public Instant timestamp() { - return context.timestamp(); - } - - @Override - public PaneInfo pane() { - return context.pane(); - } - - @Override - public BoundedWindow window() { - return context.window(); - } - - @Override - public DoFn.InputProvider inputProvider() { - throw new UnsupportedOperationException("inputProvider() exists only for testing"); - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - throw new UnsupportedOperationException("outputReceiver() exists only for testing"); - } - } - - public static Class getDoFnClass(OldDoFn fn) { - if (fn instanceof SimpleDoFnAdapter) { - return ((SimpleDoFnAdapter) fn).fn.getClass(); - } else { - return fn.getClass(); - } - } - - private static class SimpleDoFnAdapter extends OldDoFn { - - private final DoFn fn; - private transient DoFnInvoker invoker; - - private SimpleDoFnAdapter(DoFnReflector reflector, DoFn fn) { - super(fn.aggregators); - this.fn = fn; - this.invoker = reflector.bindInvoker(fn); - } - - @Override - public void setup() throws Exception { - invoker.invokeSetup(); - } - - @Override - public void startBundle(OldDoFn.Context c) throws Exception { - ContextAdapter adapter = new ContextAdapter<>(fn, c); - invoker.invokeStartBundle(adapter, adapter); - } - - @Override - public void finishBundle(OldDoFn.Context c) throws Exception { - ContextAdapter adapter = new ContextAdapter<>(fn, c); - invoker.invokeFinishBundle(adapter, adapter); - } - - @Override - public void teardown() { - invoker.invokeTeardown(); - } - - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception { - ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); - invoker.invokeProcessElement(adapter, adapter); - } - - @Override - protected TypeDescriptor getInputTypeDescriptor() { - return fn.getInputTypeDescriptor(); - } - - @Override - protected TypeDescriptor getOutputTypeDescriptor() { - return fn.getOutputTypeDescriptor(); + if (matches.size() == 0) { + checkArgument( + !required, + "No method annotated with @%s found in %s", + anno.getSimpleName(), + fnClazz.getName()); + return null; } - @Override - public Duration getAllowedTimestampSkew() { - return fn.getAllowedTimestampSkew(); + // If we have at least one match, then either it should be the only match + // or it should be an extension of the other matches (which came from parent + // classes). + Method first = matches.iterator().next(); + for (Method other : matches) { + checkArgument( + first.getName().equals(other.getName()) + && Arrays.equals(first.getParameterTypes(), other.getParameterTypes()), + "Found multiple methods annotated with @%s. [%s] and [%s]", + anno.getSimpleName(), + format(first), + format(other)); } - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.include(fn); - } + // We need to be able to call it. We require it is public. + checkArgument( + (first.getModifiers() & Modifier.PUBLIC) != 0, "%s must be public", format(first)); - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - invoker = DoFnReflector.of(fn.getClass()).bindInvoker(fn); - } - } + // And make sure its not static. + checkArgument( + (first.getModifiers() & Modifier.STATIC) == 0, "%s must not be static", format(first)); - private static class WindowDoFnAdapter - extends SimpleDoFnAdapter implements OldDoFn.RequiresWindowAccess { - - private WindowDoFnAdapter(DoFnReflector reflector, DoFn fn) { - super(reflector, fn); - } + return first; } - private static enum BeforeDelegation { - NOOP { - @Override - StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep) { - Preconditions.checkArgument(!finalStep, - "Shouldn't use NOOP delegation if there is nothing to do afterwards."); - return StackManipulation.Trivial.INSTANCE; - } - }, - INVOKE_PREPARE_FOR_PROCESSING { - private final Assigner assigner = Assigner.DEFAULT; - - @Override - StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep) { - MethodDescription prepareMethod; - try { - prepareMethod = new MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod( - DoFn.class.getDeclaredMethod("prepareForProcessing"))) - .resolve(instrumentedMethod); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException("Unable to locate prepareForProcessing method", e); - } - - if (finalStep) { - return new StackManipulation.Compound( - // Invoke the prepare method - MethodInvoker.Simple.INSTANCE.invoke(prepareMethod), - // Return from the invokeStartBundle when we're done. - TerminationHandler.Returning.INSTANCE.resolve( - assigner, instrumentedMethod, prepareMethod)); - } else { - return new StackManipulation.Compound( - // Duplicate the delegation target so that it remains after we invoke prepare - Duplication.duplicate(delegateType), - // Invoke the prepare method - MethodInvoker.Simple.INSTANCE.invoke(prepareMethod), - // Drop the return value from prepareForProcessing - TerminationHandler.Dropping.INSTANCE.resolve( - assigner, instrumentedMethod, prepareMethod)); - } - } - }; - - /** - * Stack manipulation to perform prior to the delegate call. - * - *
    - *
  • Precondition: Stack has the delegate target on top of the stack - *
  • Postcondition: If finalStep is true, then we've returned from the method. Otherwise, the - * stack still has the delegate target on top of the stack. - *
- * - * @param delegateType The type of the delegate target, in case it needs to be duplicated. - * @param instrumentedMethod The method bing instrumented. Necessary for resolving types and - * other information. - * @param finalStep If true, return from the {@code invokeStartBundle} method after invoking - * {@code prepareForProcessing} on the delegate. - */ - abstract StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep); - } - - /** - * A byte-buddy {@link Implementation} that delegates a call that receives - * {@link AdditionalParameter} to the given {@link DoFn} method. - */ - private static final class InvokerDelegation implements Implementation { - @Nullable - private final Method target; - private final BeforeDelegation before; - private final List args; - private final Assigner assigner = Assigner.DEFAULT; - private FieldDescription field; - - /** - * Create the {@link InvokerDelegation} for the specified method. - * - * @param target the method to delegate to - * @param isStartBundle whether or not this is the {@code startBundle} call - * @param args the {@link AdditionalParameter} to be passed to the {@code target} - */ - private InvokerDelegation( - @Nullable Method target, - BeforeDelegation before, - List args) { - this.target = target; - this.before = before; - this.args = args; - } - - /** - * Generate the {@link Implementation} of one of the life-cycle methods of a - * {@link DoFn}. - */ - private static Implementation create( - @Nullable final Method target, BeforeDelegation before, List args) { - if (target == null && before == BeforeDelegation.NOOP) { - // There is no target to call and nothing needs to happen before. Just produce a stub. - return StubMethod.INSTANCE; - } else { - // We need to generate a non-empty method implementation. - return new InvokerDelegation(target, before, args); - } - } - - @Override - public InstrumentedType prepare(InstrumentedType instrumentedType) { - // Remember the field description of the instrumented type. - field = instrumentedType.getDeclaredFields() - .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)).getOnly(); - - // Delegating the method call doesn't require any changes to the instrumented type. - return instrumentedType; - } - - /** - * Stack manipulation to push the {@link DoFn} reference stored in the - * delegate field of the invoker on to the top of the stack. - * - *

This implementation is derived from the code for - * {@code MethodCall.invoke(m).onInstanceField(clazz, delegateField)} with two key differences. - * First, it doesn't add a synthetic field each time, which is critical to avoid duplicate field - * definitions. Second, it uses the {@link AdditionalParameter} to populate the arguments to the - * method. - */ - private StackManipulation pushDelegateField() { - return new StackManipulation.Compound( - // Push "this" reference to the stack - MethodVariableAccess.REFERENCE.loadOffset(0), - // Access the delegate field of the the invoker - FieldAccess.forField(field).getter()); - } - - private StackManipulation pushArgument( - AdditionalParameter arg, MethodDescription instrumentedMethod) { - MethodDescription transform = arg.method; - - return new StackManipulation.Compound( - // Push the ExtraContextFactory which must have been argument 2 of the instrumented method - MethodVariableAccess.REFERENCE.loadOffset(2), - // Invoke the appropriate method to produce the context argument - MethodInvocation.invoke(transform)); - } - - private StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { - MethodDescription targetMethod = new MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod(target)).resolve(instrumentedMethod); - ParameterList params = targetMethod.getParameters(); - - List parameters; - if (!params.isEmpty()) { - // Instructions to setup the parameters for the call - parameters = new ArrayList<>(args.size() + 1); - // 1. The first argument in the delegate method must be the context. This corresponds to - // the first argument in the instrumented method, so copy that. - parameters.add(MethodVariableAccess.of(params.get(0).getType().getSuperClass()) - .loadOffset(1)); - // 2. For each of the extra arguments push the appropriate value. - for (AdditionalParameter arg : args) { - parameters.add(pushArgument(arg, instrumentedMethod)); - } - } else { - parameters = Collections.emptyList(); - } - - return new StackManipulation.Compound( - // Push the parameters - new StackManipulation.Compound(parameters), - // Invoke the target method - wrapWithUserCodeException(MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), - // Return from the instrumented method - TerminationHandler.Returning.INSTANCE.resolve( - assigner, instrumentedMethod, targetMethod)); - } - - /** - * Wrap a given stack manipulation in a try catch block. Any exceptions thrown within the - * try are wrapped with a {@link UserCodeException}. - */ - private StackManipulation wrapWithUserCodeException( - final StackManipulation tryBody) { - final MethodDescription createUserCodeException; - try { - createUserCodeException = new MethodDescription.ForLoadedMethod( - UserCodeException.class.getDeclaredMethod("wrap", Throwable.class)); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException("Unable to find UserCodeException.wrap", e); - } - - return new StackManipulation() { - @Override - public boolean isValid() { - return tryBody.isValid(); - } - - @Override - public Size apply(MethodVisitor mv, Context implementationContext) { - Label tryBlockStart = new Label(); - Label tryBlockEnd = new Label(); - Label catchBlockStart = new Label(); - Label catchBlockEnd = new Label(); - - String throwableName = - new TypeDescription.ForLoadedType(Throwable.class).getInternalName(); - mv.visitTryCatchBlock(tryBlockStart, tryBlockEnd, catchBlockStart, throwableName); - - // The try block attempts to perform the expected operations, then jumps to success - mv.visitLabel(tryBlockStart); - Size trySize = tryBody.apply(mv, implementationContext); - mv.visitJumpInsn(Opcodes.GOTO, catchBlockEnd); - mv.visitLabel(tryBlockEnd); - - // The handler wraps the exception, and then throws. - mv.visitLabel(catchBlockStart); - // Add the exception to the frame - mv.visitFrame(Opcodes.F_SAME1, - // No local variables - 0, new Object[] {}, - // 1 stack element (the throwable) - 1, new Object[] { throwableName }); - - Size catchSize = new StackManipulation.Compound( - MethodInvocation.invoke(createUserCodeException), - Throw.INSTANCE) - .apply(mv, implementationContext); - - mv.visitLabel(catchBlockEnd); - // The frame contents after the try/catch block is the same - // as it was before. - mv.visitFrame(Opcodes.F_SAME, - // No local variables - 0, new Object[] {}, - // No new stack variables - 0, new Object[] {}); - - return new Size( - trySize.getSizeImpact(), - Math.max(trySize.getMaximalSize(), catchSize.getMaximalSize())); - } - }; - } - - @Override - public ByteCodeAppender appender(final Target implementationTarget) { - return new ByteCodeAppender() { - @Override - public Size apply( - MethodVisitor methodVisitor, - Context implementationContext, - MethodDescription instrumentedMethod) { - StackManipulation.Size size = new StackManipulation.Compound( - // Put the target on top of the stack - pushDelegateField(), - // Do any necessary pre-delegation work - before.manipulation(field.getType().asErasure(), instrumentedMethod, target == null), - // Invoke the target method, if there is one. If there wasn't, then isStartBundle was - // true, and we've already emitted the appropriate return instructions. - target != null - ? invokeTargetMethod(instrumentedMethod) - : StackManipulation.Trivial.INSTANCE) - .apply(methodVisitor, implementationContext); - return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); - } - }; - } + private static String format(Method m) { + return ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(m); } - /** - * A constructor {@link Implementation} for a {@link DoFnInvoker class}. Produces the byte code - * for a constructor that takes a single argument and assigns it to the delegate field. - * {@link AdditionalParameter} to the given {@link DoFn} method. - */ - private static final class InvokerConstructor implements Implementation { - @Override - public InstrumentedType prepare(InstrumentedType instrumentedType) { - return instrumentedType; - } - - @Override - public ByteCodeAppender appender(final Target implementationTarget) { - return new ByteCodeAppender() { - @Override - public Size apply( - MethodVisitor methodVisitor, - Context implementationContext, - MethodDescription instrumentedMethod) { - StackManipulation.Size size = new StackManipulation.Compound( - // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), - // Invoke the super constructor (default constructor of Object) - MethodInvocation - .invoke(new TypeDescription.ForLoadedType(Object.class) - .getDeclaredMethods() - .filter(ElementMatchers.isConstructor() - .and(ElementMatchers.takesArguments(0))) - .getOnly()), - // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), - // Load the delegate argument - MethodVariableAccess.REFERENCE.loadOffset(1), - // Assign the delegate argument to the delegate field - FieldAccess.forField(implementationTarget.getInstrumentedType() - .getDeclaredFields() - .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) - .getOnly()).putter(), - // Return void. - MethodReturn.VOID - ).apply(methodVisitor, implementationContext); - return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); - } - }; - } + private static String formatType(TypeToken t) { + return ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(t.getType()); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java new file mode 100644 index 000000000000..be04f3b5d619 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java @@ -0,0 +1,148 @@ +/* + * 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 com.google.common.reflect.TypeToken; +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.List; +import javax.annotation.Nullable; + +/** + * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra + * context it requires, types of the input and output elements, etc. + * + *

See A new DoFn. + */ +class DoFnSignature { + private final Class fnClass; + private final TypeToken inputT; + private final TypeToken outputT; + private final ProcessElementMethod processElement; + @Nullable private final BundleMethod startBundle; + @Nullable private final BundleMethod finishBundle; + @Nullable private final LifecycleMethod setup; + @Nullable private final LifecycleMethod teardown; + + DoFnSignature( + Class fnClass, + TypeToken inputT, + TypeToken outputT, + ProcessElementMethod processElement, + @Nullable BundleMethod startBundle, + @Nullable BundleMethod finishBundle, + @Nullable LifecycleMethod setup, + @Nullable LifecycleMethod teardown) { + this.fnClass = fnClass; + this.inputT = inputT; + this.outputT = outputT; + this.processElement = processElement; + this.startBundle = startBundle; + this.finishBundle = finishBundle; + this.setup = setup; + this.teardown = teardown; + } + + public Class getFnClass() { + return fnClass; + } + + public TypeToken getInputT() { + return inputT; + } + + public TypeToken getOutputT() { + return outputT; + } + + public ProcessElementMethod getProcessElement() { + return processElement; + } + + @Nullable + public BundleMethod getStartBundle() { + return startBundle; + } + + @Nullable + public BundleMethod getFinishBundle() { + return finishBundle; + } + + @Nullable + public LifecycleMethod getSetup() { + return setup; + } + + @Nullable + public LifecycleMethod getTeardown() { + return teardown; + } + + static class DoFnMethod { + /** The relevant method in the user's class. */ + private final Method targetMethod; + + DoFnMethod(Method targetMethod) { + this.targetMethod = targetMethod; + } + + public Method getTargetMethod() { + return targetMethod; + } + } + + /** Describes a {@link DoFn.ProcessElement} method. */ + static class ProcessElementMethod extends DoFnMethod { + enum Parameter { + BOUNDED_WINDOW, + INPUT_PROVIDER, + OUTPUT_RECEIVER + } + + private final List extraParameters; + + ProcessElementMethod(Method targetMethod, List extraParameters) { + super(targetMethod); + this.extraParameters = Collections.unmodifiableList(extraParameters); + } + + List getExtraParameters() { + return extraParameters; + } + + /** @return true if the reflected {@link DoFn} uses a Single Window. */ + public boolean usesSingleWindow() { + return extraParameters.contains(Parameter.BOUNDED_WINDOW); + } + } + + /** Describes a {@link DoFn.StartBundle} or {@link DoFn.FinishBundle} method. */ + static class BundleMethod extends DoFnMethod { + BundleMethod(Method targetMethod) { + super(targetMethod); + } + } + + /** Describes a {@link DoFn.Setup} or {@link DoFn.Teardown} method. */ + static class LifecycleMethod extends DoFnMethod { + LifecycleMethod(Method targetMethod) { + super(targetMethod); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index dd1baab4a265..4cd410a19597 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -92,7 +92,7 @@ public static DoFnTester of(OldDoFn DoFnTester of(DoFn fn) { - return new DoFnTester(DoFnReflector.of(fn.getClass()).toDoFn(fn)); + return new DoFnTester(DoFnAdapters.toOldDoFn(fn)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index aa5753197b1d..af500ba0c9c3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -536,7 +536,7 @@ private static Bound of( private static OldDoFn adapt(DoFn fn) { - return DoFnReflector.of(fn.getClass()).toDoFn(fn); + return DoFnAdapters.toOldDoFn(fn); } /** @@ -747,7 +747,7 @@ protected Coder getDefaultOutputCoder(PCollection inp @Override protected String getKindString() { - Class clazz = DoFnReflector.getDoFnClass(fn); + Class clazz = DoFnAdapters.getDoFnClass(fn); if (clazz.isAnonymousClass()) { return "AnonymousParDo"; } else { @@ -968,7 +968,7 @@ public Coder getDefaultOutputCoder( @Override protected String getKindString() { - Class clazz = DoFnReflector.getDoFnClass(fn); + Class clazz = DoFnAdapters.getDoFnClass(fn); if (clazz.isAnonymousClass()) { return "AnonymousParMultiDo"; } else { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java new file mode 100644 index 000000000000..22d1d2794876 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java @@ -0,0 +1,474 @@ +/* + * 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.junit.Assert.assertFalse; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.transforms.dofnreflector.DoFnInvokersTestHelper; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.UserCodeException; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Tests for {@link DoFnInvokers}. */ +public class DoFnInvokersTest { + /** A convenience struct holding flags that indicate whether a particular method was invoked. */ + public static class Invocations { + public boolean wasProcessElementInvoked = false; + public boolean wasStartBundleInvoked = false; + public boolean wasFinishBundleInvoked = false; + public boolean wasSetupInvoked = false; + public boolean wasTeardownInvoked = false; + private final String name; + + public Invocations(String name) { + this.name = name; + } + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Mock private DoFn.ProcessContext mockContext; + @Mock private BoundedWindow mockWindow; + @Mock private DoFn.InputProvider mockInputProvider; + @Mock private DoFn.OutputReceiver mockOutputReceiver; + + private DoFn.ExtraContextFactory extraContextFactory; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + this.extraContextFactory = + new DoFn.ExtraContextFactory() { + @Override + public BoundedWindow window() { + return mockWindow; + } + + @Override + public DoFn.InputProvider inputProvider() { + return mockInputProvider; + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + return mockOutputReceiver; + } + }; + } + + private void checkInvokeProcessElementWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called processElement on " + invocation.name, + invocation.wasProcessElementInvoked); + } + DoFnReflector.newByteBuddyInvoker(fn).invokeProcessElement(mockContext, extraContextFactory); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called processElement on " + invocation.name, + invocation.wasProcessElementInvoked); + } + } + + private void checkInvokeStartBundleWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called startBundle on " + invocation.name, + invocation.wasStartBundleInvoked); + } + DoFnReflector.newByteBuddyInvoker(fn).invokeStartBundle(mockContext); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called startBundle on " + invocation.name, invocation.wasStartBundleInvoked); + } + } + + private void checkInvokeFinishBundleWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called finishBundle on " + invocation.name, + invocation.wasFinishBundleInvoked); + } + DoFnReflector.newByteBuddyInvoker(fn).invokeFinishBundle(mockContext); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called finishBundle on " + invocation.name, + invocation.wasFinishBundleInvoked); + } + } + + private void checkInvokeSetupWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called setup on " + invocation.name, invocation.wasSetupInvoked); + } + DoFnReflector.newByteBuddyInvoker(fn).invokeSetup(); + for (Invocations invocation : invocations) { + assertTrue("Should have called setup on " + invocation.name, invocation.wasSetupInvoked); + } + } + + private void checkInvokeTeardownWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + DoFnReflector.newByteBuddyInvoker(fn).invokeTeardown(); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called teardown on " + invocation.name, invocation.wasTeardownInvoked); + } + } + + @Test + public void testDoFnWithNoExtraContext() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + }; + + assertFalse( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnInvokersReused() throws Exception { + // Ensures that we don't create a new Invoker class for every instance of the OldDoFn. + IdentityParent fn1 = new IdentityParent(); + IdentityParent fn2 = new IdentityParent(); + assertSame( + "Invoker classes should only be generated once for each type", + DoFnReflector.newByteBuddyInvoker(fn1).getClass(), + DoFnReflector.newByteBuddyInvoker(fn2).getClass()); + } + + interface InterfaceWithProcessElement { + @DoFn.ProcessElement + void processElement(DoFn.ProcessContext c); + } + + interface LayersOfInterfaces extends InterfaceWithProcessElement {} + + private class IdentityUsingInterfaceWithProcessElement extends DoFn + implements LayersOfInterfaces { + + private Invocations invocations = new Invocations("Named Class"); + + @Override + public void processElement(DoFn.ProcessContext c) { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + } + + @Test + public void testDoFnWithProcessElementInterface() throws Exception { + IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); + assertFalse( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.invocations); + } + + private class IdentityParent extends DoFn { + protected Invocations parentInvocations = new Invocations("IdentityParent"); + + @ProcessElement + public void process(ProcessContext c) { + parentInvocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + } + + private class IdentityChildWithoutOverride extends IdentityParent {} + + private class IdentityChildWithOverride extends IdentityParent { + protected Invocations childInvocations = new Invocations("IdentityChildWithOverride"); + + @Override + public void process(DoFn.ProcessContext c) { + super.process(c); + childInvocations.wasProcessElementInvoked = true; + } + } + + @Test + public void testDoFnWithMethodInSuperclass() throws Exception { + IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); + assertFalse( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.parentInvocations); + } + + @Test + public void testDoFnWithMethodInSubclass() throws Exception { + IdentityChildWithOverride fn = new IdentityChildWithOverride(); + assertFalse( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.parentInvocations, fn.childInvocations); + } + + @Test + public void testDoFnWithWindow() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow w) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(w, mockWindow); + } + }; + + assertTrue( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithOutputReceiver() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, OutputReceiver o) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(o, mockOutputReceiver); + } + }; + + assertFalse( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithInputProvider() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, InputProvider i) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(i, mockInputProvider); + } + }; + + assertFalse( + DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithStartBundle() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + }; + + checkInvokeStartBundleWorks(fn, invocations); + checkInvokeFinishBundleWorks(fn, invocations); + } + + @Test + public void testDoFnWithSetupTeardown() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + + @Setup + public void before() { + invocations.wasSetupInvoked = true; + } + + @Teardown + public void after() { + invocations.wasTeardownInvoked = true; + } + }; + + checkInvokeSetupWorks(fn, invocations); + checkInvokeTeardownWorks(fn, invocations); + } + + private static class PrivateDoFnClass extends DoFn { + final Invocations invocations = new Invocations(getClass().getName()); + + @ProcessElement + public void processThis(ProcessContext c) { + invocations.wasProcessElementInvoked = true; + } + } + + @Test + public void testLocalPrivateDoFnClass() throws Exception { + PrivateDoFnClass fn = new PrivateDoFnClass(); + checkInvokeProcessElementWorks(fn, fn.invocations); + } + + @Test + public void testStaticPackagePrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticPackagePrivateDoFn"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticPackagePrivateDoFn(invocations), invocations); + } + + @Test + public void testInnerPackagePrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("InnerPackagePrivateDoFn"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerPackagePrivateDoFn(invocations), invocations); + } + + @Test + public void testStaticPrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticPrivateDoFn"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticPrivateDoFn(invocations), invocations); + } + + @Test + public void testInnerPrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticInnerDoFn"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerPrivateDoFn(invocations), invocations); + } + + @Test + public void testAnonymousInnerDoFnInOtherPackage() throws Exception { + Invocations invocations = new Invocations("AnonymousInnerDoFnInOtherPackage"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerAnonymousDoFn(invocations), invocations); + } + + @Test + public void testStaticAnonymousDoFnInOtherPackage() throws Exception { + Invocations invocations = new Invocations("AnonymousStaticDoFnInOtherPackage"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticAnonymousDoFn(invocations), invocations); + } + + @Test + public void testProcessElementException() throws Exception { + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) { + throw new IllegalArgumentException("bogus"); + } + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnReflector.newByteBuddyInvoker(fn).invokeProcessElement(null, null); + } + + @Test + public void testStartBundleException() throws Exception { + DoFn fn = + new DoFn() { + @StartBundle + public void startBundle(@SuppressWarnings("unused") Context c) { + throw new IllegalArgumentException("bogus"); + } + + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnReflector.newByteBuddyInvoker(fn).invokeStartBundle(null); + } + + @Test + public void testFinishBundleException() throws Exception { + DoFn fn = + new DoFn() { + @FinishBundle + public void finishBundle(@SuppressWarnings("unused") Context c) { + throw new IllegalArgumentException("bogus"); + } + + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnReflector.newByteBuddyInvoker(fn).invokeFinishBundle(null); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java index e05e5e2c667a..0d8cfb0ca404 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java @@ -17,553 +17,40 @@ */ package org.apache.beam.sdk.transforms; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; - -import org.apache.beam.sdk.transforms.DoFn.Context; -import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.transforms.dofnreflector.DoFnReflectorTestHelper; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.UserCodeException; - -import org.junit.Before; +import com.google.common.reflect.TypeToken; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; import java.lang.reflect.Method; +import java.util.List; -/** - * Tests for {@link DoFnReflector}. - */ +/** Tests for {@link DoFnReflector}. */ @RunWith(JUnit4.class) public class DoFnReflectorTest { - /** - * A convenience struct holding flags that indicate whether a particular method was invoked. - */ - public static class Invocations { - public boolean wasProcessElementInvoked = false; - public boolean wasStartBundleInvoked = false; - public boolean wasFinishBundleInvoked = false; - public boolean wasSetupInvoked = false; - public boolean wasTeardownInvoked = false; - private final String name; - - public Invocations(String name) { - this.name = name; - } - } - - private DoFn fn; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Mock - private DoFn.ProcessContext mockContext; - @Mock - private BoundedWindow mockWindow; - @Mock - private DoFn.InputProvider mockInputProvider; - @Mock - private DoFn.OutputReceiver mockOutputReceiver; - - private ExtraContextFactory extraContextFactory; - - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - this.extraContextFactory = new ExtraContextFactory() { - @Override - public BoundedWindow window() { - return mockWindow; - } - - @Override - public DoFn.InputProvider inputProvider() { - return mockInputProvider; - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - return mockOutputReceiver; - } - }; - } - - private DoFnReflector underTest(DoFn fn) { - this.fn = fn; - return DoFnReflector.of(fn.getClass()); - } - - private void checkInvokeProcessElementWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called processElement on " + invocation.name, - invocation.wasProcessElementInvoked); - } - r.bindInvoker(fn).invokeProcessElement(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called processElement on " + invocation.name, - invocation.wasProcessElementInvoked); - } - } - - private void checkInvokeStartBundleWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called startBundle on " + invocation.name, - invocation.wasStartBundleInvoked); - } - r.bindInvoker(fn).invokeStartBundle(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called startBundle on " + invocation.name, - invocation.wasStartBundleInvoked); - } - } - - private void checkInvokeFinishBundleWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called finishBundle on " + invocation.name, - invocation.wasFinishBundleInvoked); - } - r.bindInvoker(fn).invokeFinishBundle(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called finishBundle on " + invocation.name, - invocation.wasFinishBundleInvoked); - } - } - - private void checkInvokeSetupWorks(DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called setup on " + invocation.name, - invocation.wasSetupInvoked); - } - r.bindInvoker(fn).invokeSetup(); - for (Invocations invocation : invocations) { - assertTrue("Should have called setup on " + invocation.name, - invocation.wasSetupInvoked); - } - } - - private void checkInvokeTeardownWorks(DoFnReflector r, Invocations... invocations) - throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called teardown on " + invocation.name, - invocation.wasTeardownInvoked); - } - r.bindInvoker(fn).invokeTeardown(); - for (Invocations invocation : invocations) { - assertTrue("Should have called teardown on " + invocation.name, - invocation.wasTeardownInvoked); - } - } - - @Test - public void testDoFnWithNoExtraContext() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnInvokersReused() throws Exception { - // Ensures that we don't create a new Invoker class for every instance of the OldDoFn. - IdentityParent fn1 = new IdentityParent(); - IdentityParent fn2 = new IdentityParent(); - DoFnReflector reflector1 = underTest(fn1); - DoFnReflector reflector2 = underTest(fn2); - assertSame("DoFnReflector instances should be cached and reused for identical types", - reflector1, reflector2); - assertSame("Invoker classes should only be generated once for each type", - reflector1.bindInvoker(fn1).getClass(), - reflector2.bindInvoker(fn2).getClass()); - } - - interface InterfaceWithProcessElement { - @ProcessElement - void processElement(DoFn.ProcessContext c); - } - - interface LayersOfInterfaces extends InterfaceWithProcessElement {} - - private class IdentityUsingInterfaceWithProcessElement - extends DoFn - implements LayersOfInterfaces { - - private Invocations invocations = new Invocations("Named Class"); - - @Override - public void processElement(DoFn.ProcessContext c) { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - } - - @Test - public void testDoFnWithProcessElementInterface() throws Exception { - IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.invocations); - } - - private class IdentityParent extends DoFn { - protected Invocations parentInvocations = new Invocations("IdentityParent"); - - @ProcessElement - public void process(ProcessContext c) { - parentInvocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - } - - private class IdentityChildWithoutOverride extends IdentityParent { - } - - private class IdentityChildWithOverride extends IdentityParent { - protected Invocations childInvocations = new Invocations("IdentityChildWithOverride"); - - @Override - public void process(DoFn.ProcessContext c) { - super.process(c); - childInvocations.wasProcessElementInvoked = true; - } - } - - @Test - public void testDoFnWithMethodInSuperclass() throws Exception { - IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.parentInvocations); - } - - @Test - public void testDoFnWithMethodInSubclass() throws Exception { - IdentityChildWithOverride fn = new IdentityChildWithOverride(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.parentInvocations, fn.childInvocations); - } - - @Test - public void testDoFnWithWindow() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow w) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(w, mockWindow); - } - }); - - assertTrue(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithOutputReceiver() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, DoFn.OutputReceiver o) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(o, mockOutputReceiver); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithInputProvider() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, DoFn.InputProvider i) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(i, mockInputProvider); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithStartBundle() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) {} - - @StartBundle - public void startBundle(Context c) { - invocations.wasStartBundleInvoked = true; - assertSame(c, mockContext); - } - - @FinishBundle - public void finishBundle(Context c) { - invocations.wasFinishBundleInvoked = true; - assertSame(c, mockContext); - } - }); - - checkInvokeStartBundleWorks(reflector, invocations); - checkInvokeFinishBundleWorks(reflector, invocations); - } - - @Test - public void testDoFnWithSetupTeardown() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) {} - - @StartBundle - public void startBundle(Context c) { - invocations.wasStartBundleInvoked = true; - assertSame(c, mockContext); - } - - @FinishBundle - public void finishBundle(Context c) { - invocations.wasFinishBundleInvoked = true; - assertSame(c, mockContext); - } - - @Setup - public void before() { - invocations.wasSetupInvoked = true; - } - - @Teardown - public void after() { - invocations.wasTeardownInvoked = true; - } - }); - - checkInvokeSetupWorks(reflector, invocations); - checkInvokeTeardownWorks(reflector, invocations); - } - - @Test - public void testNoProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("No method annotated with @ProcessElement found"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() {}); - } - - @Test - public void testMultipleProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @ProcessElement"); - thrown.expectMessage("foo()"); - thrown.expectMessage("bar()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @ProcessElement - public void bar() {} - }); - } - - @Test - public void testMultipleStartBundleElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @StartBundle"); - thrown.expectMessage("bar()"); - thrown.expectMessage("baz()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @StartBundle - public void bar() {} + @Rule public ExpectedException thrown = ExpectedException.none(); - @StartBundle - public void baz() {} - }); - } - - @Test - public void testMultipleFinishBundleElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); - thrown.expectMessage("bar()"); - thrown.expectMessage("baz()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @FinishBundle - public void bar() {} - - @FinishBundle - public void baz() {} - }); - } - - private static class PrivateDoFnClass extends DoFn { - final Invocations invocations = new Invocations(getClass().getName()); - - @ProcessElement - public void processThis(ProcessContext c) { - invocations.wasProcessElementInvoked = true; - } - } - - @Test - public void testLocalPrivateDoFnClass() throws Exception { - PrivateDoFnClass fn = new PrivateDoFnClass(); - DoFnReflector reflector = underTest(fn); - checkInvokeProcessElementWorks(reflector, fn.invocations); - } - - @Test - public void testStaticPackagePrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticPackagePrivateDoFn"); - DoFnReflector reflector = - underTest(DoFnReflectorTestHelper.newStaticPackagePrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testInnerPackagePrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("InnerPackagePrivateDoFn"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerPackagePrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testStaticPrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticPrivateDoFn"); - DoFnReflector reflector = underTest(DoFnReflectorTestHelper.newStaticPrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testInnerPrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticInnerDoFn"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerPrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testAnonymousInnerDoFnInOtherPackage() throws Exception { - Invocations invocations = new Invocations("AnonymousInnerDoFnInOtherPackage"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerAnonymousDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testStaticAnonymousDoFnInOtherPackage() throws Exception { - Invocations invocations = new Invocations("AnonymousStaticDoFnInOtherPackage"); - DoFnReflector reflector = - underTest(DoFnReflectorTestHelper.newStaticAnonymousDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testPrivateProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("process() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - private void process() {} - }); - } - - @Test - public void testPrivateStartBundle() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("startBundle() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void processElement() {} - - @StartBundle - void startBundle() {} - }); - } - - @Test - public void testPrivateFinishBundle() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("finishBundle() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void processElement() {} - - @FinishBundle - void finishBundle() {} - }); - } + private static class FakeDoFn extends DoFn {} @SuppressWarnings({"unused"}) private void missingProcessContext() {} @Test public void testMissingProcessContext() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() - + "#missingProcessContext() must take a ProcessContext as its first argument"); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#missingProcessContext() must take a ProcessContext<> as its first argument"); - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("missingProcessContext")); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("missingProcessContext"), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } @SuppressWarnings({"unused"}) @@ -571,12 +58,16 @@ private void badProcessContext(String s) {} @Test public void testBadProcessContextType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() - + "#badProcessContext(String) must take a ProcessContext as its first argument"); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#badProcessContext(String) must take a ProcessContext<> as its first argument"); - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("badProcessContext", String.class)); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badProcessContext", String.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } @SuppressWarnings({"unused"}) @@ -584,30 +75,36 @@ private void badExtraContext(DoFn.Context c, int n) {} @Test public void testBadExtraContext() throws Exception { - thrown.expect(IllegalStateException.class); + thrown.expect(IllegalArgumentException.class); thrown.expectMessage( - "int is not a valid context parameter for method " - + getClass().getName() + "#badExtraContext(Context, int). Should be one of ["); + getClass().getName() + + "#badExtraContext(Context, int) must have a single argument of type Context"); - DoFnReflector.verifyBundleMethodArguments( - getClass().getDeclaredMethod("badExtraContext", Context.class, int.class)); + DoFnReflector.analyzeBundleMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badExtraContext", DoFn.Context.class, int.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } @SuppressWarnings({"unused"}) - private void badExtraProcessContext( - DoFn.ProcessContext c, Integer n) {} + private void badExtraProcessContext(DoFn.ProcessContext c, Integer n) {} @Test public void testBadExtraProcessContextType() throws Exception { - thrown.expect(IllegalStateException.class); + thrown.expect(IllegalArgumentException.class); thrown.expectMessage( "Integer is not a valid context parameter for method " - + getClass().getName() + "#badExtraProcessContext(ProcessContext, Integer)" - + ". Should be one of [BoundedWindow]"); + + getClass().getName() + + "#badExtraProcessContext(ProcessContext, Integer)" + + ". Should be one of [BoundedWindow]"); - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("badExtraProcessContext", - ProcessContext.class, Integer.class)); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass() + .getDeclaredMethod("badExtraProcessContext", DoFn.ProcessContext.class, Integer.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } @SuppressWarnings("unused") @@ -617,82 +114,65 @@ private int badReturnType() { @Test public void testBadReturnType() throws Exception { - thrown.expect(IllegalStateException.class); + thrown.expect(IllegalArgumentException.class); thrown.expectMessage(getClass().getName() + "#badReturnType() must have a void return type"); - DoFnReflector.verifyProcessMethodArguments(getClass().getDeclaredMethod("badReturnType")); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badReturnType"), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } @SuppressWarnings("unused") - private void goodGenerics( + private void goodConcreteTypes( DoFn.ProcessContext c, DoFn.InputProvider input, DoFn.OutputReceiver output) {} @Test - public void testValidGenerics() throws Exception { + public void testGoodConcreteTypes() throws Exception { Method method = getClass() .getDeclaredMethod( - "goodGenerics", + "goodConcreteTypes", DoFn.ProcessContext.class, DoFn.InputProvider.class, DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } - @SuppressWarnings("unused") - private void goodWildcards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} + private static class GoodTypeVariables extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void goodTypeVariables( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + } @Test - public void testGoodWildcards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodWildcards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); + public void testGoodTypeVariables() throws Exception { + DoFnReflector.getOrParseSignature(GoodTypeVariables.class); } - @SuppressWarnings("unused") - private void goodBoundedWildcards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodBoundedWildcards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodBoundedWildcards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); + private static class IdentityFn extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void processElement(ProcessContext c, InputProvider input, OutputReceiver output) { + c.output(c.element()); + } } - @SuppressWarnings("unused") - private void goodTypeVariables( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} + private static class IdentityListFn extends IdentityFn> {} @Test - public void testGoodTypeVariables() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodTypeVariables", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); + public void testIdentityFnApplied() throws Exception { + DoFnReflector.getOrParseSignature(new IdentityFn() {}.getClass()); } @SuppressWarnings("unused") @@ -711,14 +191,20 @@ public void testBadGenericsTwoArgs() throws Exception { DoFn.InputProvider.class, DoFn.OutputReceiver.class); - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver " - + "for method " + getClass().getName() - + "#badGenericTwoArgs(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter " + + "for method " + + getClass().getName() + + "#badGenericTwoArgs(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); - DoFnReflector.verifyProcessMethodArguments(method); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } @SuppressWarnings("unused") @@ -737,86 +223,147 @@ public void testBadGenericWildCards() throws Exception { DoFn.InputProvider.class, DoFn.OutputReceiver.class); - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver for method " - + getClass().getName() - + "#badGenericWildCards(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter for method " + + getClass().getName() + + "#badGenericWildCards(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); - DoFnReflector.verifyProcessMethodArguments(method); + DoFnReflector.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); } - @SuppressWarnings("unused") - private void badTypeVariables(DoFn.ProcessContext c, - DoFn.InputProvider input, DoFn.OutputReceiver output) {} + static class BadTypeVariables extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void badTypeVariables( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + } @Test public void testBadTypeVariables() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badTypeVariables", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter for method " + + BadTypeVariables.class.getName() + + "#badTypeVariables(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver for method " + getClass().getName() - + "#badTypeVariables(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); + DoFnReflector.getOrParseSignature(BadTypeVariables.class); + } - DoFnReflector.verifyProcessMethodArguments(method); + @Test + public void testNoProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("No method annotated with @ProcessElement found"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature(new DoFn() {}.getClass()); } @Test - public void testProcessElementException() throws Exception { - DoFn fn = new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - throw new IllegalArgumentException("bogus"); - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeProcessElement(null, null); + public void testMultipleProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @ProcessElement"); + thrown.expectMessage("foo()"); + thrown.expectMessage("bar()"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo() {} + + @ProcessElement + public void bar() {} + }.getClass()); } @Test - public void testStartBundleException() throws Exception { - DoFn fn = new DoFn() { - @StartBundle - public void startBundle(@SuppressWarnings("unused") Context c) { - throw new IllegalArgumentException("bogus"); - } - - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeStartBundle(null, null); + public void testMultipleStartBundleElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @StartBundle"); + thrown.expectMessage("bar()"); + thrown.expectMessage("baz()"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo() {} + + @StartBundle + public void bar() {} + + @StartBundle + public void baz() {} + }.getClass()); + } + + @Test + public void testMultipleFinishBundleMethods() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); + thrown.expectMessage("bar(Context)"); + thrown.expectMessage("baz(Context)"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo(ProcessContext context) {} + + @FinishBundle + public void bar(Context context) {} + + @FinishBundle + public void baz(Context context) {} + }.getClass()); + } + + @Test + public void testPrivateProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("process() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature( + new DoFn() { + @ProcessElement + private void process() {} + }.getClass()); } @Test - public void testFinishBundleException() throws Exception { - DoFn fn = new DoFn() { - @FinishBundle - public void finishBundle(@SuppressWarnings("unused") Context c) { - throw new IllegalArgumentException("bogus"); - } - - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeFinishBundle(null, null); + public void testPrivateStartBundle() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("startBundle() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature( + new DoFn() { + @ProcessElement + public void processElement() {} + + @StartBundle + void startBundle() {} + }.getClass()); + } + + @Test + public void testPrivateFinishBundle() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("finishBundle() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnReflector.getOrParseSignature( + new DoFn() { + @ProcessElement + public void processElement() {} + + @FinishBundle + void finishBundle() {} + }.getClass()); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java index 604536bca80e..3469223c38fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java @@ -99,7 +99,7 @@ public void testFlattenPCollectionListThenParDo() { PCollection output = makePCollectionListOfStrings(p, inputs) .apply(Flatten.pCollections()) - .apply(ParDo.of(new IdentityFn(){})); + .apply(ParDo.of(new IdentityFn())); PAssert.that(output).containsInAnyOrder(flattenLists(inputs)); p.run(); @@ -152,7 +152,7 @@ public void testFlattenPCollectionListEmptyThenParDo() { PCollection output = PCollectionList.empty(p) .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()) - .apply(ParDo.of(new IdentityFn(){})); + .apply(ParDo.of(new IdentityFn())); PAssert.that(output).empty(); p.run(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnInvokersTestHelper.java similarity index 93% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnInvokersTestHelper.java index 90fba12b9599..d3ad3e521f85 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnInvokersTestHelper.java @@ -18,13 +18,14 @@ package org.apache.beam.sdk.transforms.dofnreflector; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnReflectorTest.Invocations; +import org.apache.beam.sdk.transforms.DoFnInvokersTest; +import org.apache.beam.sdk.transforms.DoFnInvokersTest.Invocations; /** - * Test helper for DoFnReflectorTest, which needs to test package-private access + * Test helper for {@link DoFnInvokersTest}, which needs to test package-private access * to DoFns in other packages. */ -public class DoFnReflectorTestHelper { +public class DoFnInvokersTestHelper { private static class StaticPrivateDoFn extends DoFn { final Invocations invocations; diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java index 91ecd162d875..8d9c344bb858 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java @@ -22,8 +22,9 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; +import org.apache.beam.sdk.transforms.DoFnAdapters; +import org.apache.beam.sdk.transforms.DoFnInvoker; import org.apache.beam.sdk.transforms.DoFnReflector; -import org.apache.beam.sdk.transforms.DoFnReflector.DoFnInvoker; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -60,16 +61,14 @@ public class DoFnReflectorBenchmark { private ExtraContextFactory extraContextFactory = new DoFn.FakeExtraContextFactory<>(); - private DoFnReflector doFnReflector; private OldDoFn adaptedDoFnWithContext; private DoFnInvoker invoker; @Setup public void setUp() { - doFnReflector = DoFnReflector.of(doFn.getClass()); - adaptedDoFnWithContext = doFnReflector.toDoFn(doFn); - invoker = doFnReflector.bindInvoker(doFn); + adaptedDoFnWithContext = DoFnAdapters.toOldDoFn(doFn); + invoker = DoFnReflector.newByteBuddyInvoker(doFn); } @Benchmark From a34917de7066a45eebc57156b47f85b942ed1756 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 16 Aug 2016 14:56:14 -0700 Subject: [PATCH 2/4] Addressed comments by bjchambers --- .../org/apache/beam/sdk/transforms/DoFn.java | 17 +- .../beam/sdk/transforms/DoFnAdapters.java | 14 +- .../beam/sdk/transforms/DoFnSignature.java | 148 ------------------ .../transforms/{ => reflect}/DoFnInvoker.java | 4 +- .../{ => reflect}/DoFnInvokers.java | 90 ++++++----- .../sdk/transforms/reflect/DoFnSignature.java | 113 +++++++++++++ .../DoFnSignatures.java} | 47 +++--- .../sdk/transforms/reflect/package-info.java | 22 +++ .../{ => reflect}/DoFnInvokersTest.java | 66 +++++--- .../DoFnInvokersTestHelper.java | 5 +- .../DoFnSignaturesTest.java} | 44 +++--- ...chmark.java => DoFnInvokersBenchmark.java} | 41 ++--- 12 files changed, 312 insertions(+), 299 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java rename sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/{ => reflect}/DoFnInvoker.java (95%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/{ => reflect}/DoFnInvokers.java (89%) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java rename sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/{DoFnReflector.java => reflect/DoFnSignatures.java} (91%) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java rename sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/{ => reflect}/DoFnInvokersTest.java (88%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/{dofnreflector => reflect}/DoFnInvokersTestHelper.java (95%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/{DoFnReflectorTest.java => reflect/DoFnSignaturesTest.java} (90%) rename sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/{DoFnReflectorBenchmark.java => DoFnInvokersBenchmark.java} (85%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 80b67afc8412..81d34c1eddef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.PCollectionView; @@ -247,7 +248,7 @@ public Duration getAllowedTimestampSkew() { ///////////////////////////////////////////////////////////////////////////// - Map> aggregators = new HashMap<>(); + protected Map> aggregators = new HashMap<>(); /** * Protects aggregators from being created after initialization. @@ -283,7 +284,7 @@ protected TypeDescriptor getOutputTypeDescriptor() { /** * Interface for runner implementors to provide implementations of extra context information. * - *

The methods on this interface are called by {@link DoFnReflector} before invoking an + *

The methods on this interface are called by {@link DoFnInvoker} before invoking an * annotated {@link StartBundle}, {@link ProcessElement} or {@link FinishBundle} method that * has indicated it needs the given extra context. * @@ -301,23 +302,23 @@ public interface ExtraContextFactory { BoundedWindow window(); /** - * A placeholder for testing purposes. The return type itself is package-private and not - * implemented. + * A placeholder for testing purposes. */ InputProvider inputProvider(); /** - * A placeholder for testing purposes. The return type itself is package-private and not - * implemented. + * A placeholder for testing purposes. */ OutputReceiver outputReceiver(); } - static interface OutputReceiver { + /** A placeholder for testing purposes. */ + public interface OutputReceiver { void output(T output); } - static interface InputProvider { + /** A placeholder for testing purposes. */ + public interface InputProvider { T get(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index 6079c184036b..6bac4274216a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -19,6 +19,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.PCollectionView; @@ -47,10 +51,10 @@ public static Class getDoFnClass(OldDoFn fn) { } } - /** Create a {@link OldDoFn} that the {@link DoFn}. */ + /** Creates an {@link OldDoFn} that delegates to the {@link DoFn}. */ public static OldDoFn toOldDoFn(DoFn fn) { - DoFnSignature signature = DoFnReflector.getOrParseSignature(fn.getClass()); - if (signature.getProcessElement().usesSingleWindow()) { + DoFnSignature signature = DoFnSignatures.INSTANCE.getOrParseSignature(fn.getClass()); + if (signature.processElement().usesSingleWindow()) { return new WindowDoFnAdapter<>(fn); } else { return new SimpleDoFnAdapter<>(fn); @@ -68,7 +72,7 @@ private static class SimpleDoFnAdapter extends OldDoFn fn) { super(fn.aggregators); this.fn = fn; - this.invoker = DoFnReflector.newByteBuddyInvoker(fn); + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); } @Override @@ -121,7 +125,7 @@ public void populateDisplayData(DisplayData.Builder builder) { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.invoker = DoFnReflector.newByteBuddyInvoker(fn); + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java deleted file mode 100644 index be04f3b5d619..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSignature.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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 com.google.common.reflect.TypeToken; -import java.lang.reflect.Method; -import java.util.Collections; -import java.util.List; -import javax.annotation.Nullable; - -/** - * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra - * context it requires, types of the input and output elements, etc. - * - *

See A new DoFn. - */ -class DoFnSignature { - private final Class fnClass; - private final TypeToken inputT; - private final TypeToken outputT; - private final ProcessElementMethod processElement; - @Nullable private final BundleMethod startBundle; - @Nullable private final BundleMethod finishBundle; - @Nullable private final LifecycleMethod setup; - @Nullable private final LifecycleMethod teardown; - - DoFnSignature( - Class fnClass, - TypeToken inputT, - TypeToken outputT, - ProcessElementMethod processElement, - @Nullable BundleMethod startBundle, - @Nullable BundleMethod finishBundle, - @Nullable LifecycleMethod setup, - @Nullable LifecycleMethod teardown) { - this.fnClass = fnClass; - this.inputT = inputT; - this.outputT = outputT; - this.processElement = processElement; - this.startBundle = startBundle; - this.finishBundle = finishBundle; - this.setup = setup; - this.teardown = teardown; - } - - public Class getFnClass() { - return fnClass; - } - - public TypeToken getInputT() { - return inputT; - } - - public TypeToken getOutputT() { - return outputT; - } - - public ProcessElementMethod getProcessElement() { - return processElement; - } - - @Nullable - public BundleMethod getStartBundle() { - return startBundle; - } - - @Nullable - public BundleMethod getFinishBundle() { - return finishBundle; - } - - @Nullable - public LifecycleMethod getSetup() { - return setup; - } - - @Nullable - public LifecycleMethod getTeardown() { - return teardown; - } - - static class DoFnMethod { - /** The relevant method in the user's class. */ - private final Method targetMethod; - - DoFnMethod(Method targetMethod) { - this.targetMethod = targetMethod; - } - - public Method getTargetMethod() { - return targetMethod; - } - } - - /** Describes a {@link DoFn.ProcessElement} method. */ - static class ProcessElementMethod extends DoFnMethod { - enum Parameter { - BOUNDED_WINDOW, - INPUT_PROVIDER, - OUTPUT_RECEIVER - } - - private final List extraParameters; - - ProcessElementMethod(Method targetMethod, List extraParameters) { - super(targetMethod); - this.extraParameters = Collections.unmodifiableList(extraParameters); - } - - List getExtraParameters() { - return extraParameters; - } - - /** @return true if the reflected {@link DoFn} uses a Single Window. */ - public boolean usesSingleWindow() { - return extraParameters.contains(Parameter.BOUNDED_WINDOW); - } - } - - /** Describes a {@link DoFn.StartBundle} or {@link DoFn.FinishBundle} method. */ - static class BundleMethod extends DoFnMethod { - BundleMethod(Method targetMethod) { - super(targetMethod); - } - } - - /** Describes a {@link DoFn.Setup} or {@link DoFn.Teardown} method. */ - static class LifecycleMethod extends DoFnMethod { - LifecycleMethod(Method targetMethod) { - super(targetMethod); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java similarity index 95% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index 6785263fb352..5818a5933f2d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -15,7 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.sdk.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; /** * Interface for invoking the {@code DoFn} processing methods. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java similarity index 89% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java index 93687856bf78..73874d767d3c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnInvokers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -15,10 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.sdk.transforms.reflect; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundle; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.DoFn.Setup; @@ -26,8 +28,6 @@ import org.apache.beam.sdk.transforms.DoFn.Teardown; import org.apache.beam.sdk.util.UserCodeException; -import com.google.common.base.Preconditions; - import net.bytebuddy.ByteBuddy; import net.bytebuddy.NamingStrategy; import net.bytebuddy.description.field.FieldDescription; @@ -65,27 +65,34 @@ import javax.annotation.Nullable; /** Dynamically generates {@link DoFnInvoker} instances for invoking a {@link DoFn}. */ -abstract class DoFnInvokers { +public class DoFnInvokers { + public static final DoFnInvokers INSTANCE = new DoFnInvokers(); + + private static final String FN_DELEGATE_FIELD_NAME = "delegate"; + /** * A cache of constructors of generated {@link DoFnInvoker} classes, keyed by {@link DoFn} class. * Needed because generating an invoker class is expensive, and to avoid generating an excessive * number of classes consuming PermGen memory. */ - private static final Map, Constructor> BYTE_BUDDY_INVOKER_CONSTRUCTOR_CACHE = + private final Map, Constructor> byteBuddyInvokerConstructorCache = new LinkedHashMap<>(); - private static final String FN_DELEGATE_FIELD_NAME = "delegate"; - - /** This is a factory class that should not be instantiated. */ private DoFnInvokers() {} /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ - static DoFnInvoker newByteBuddyInvoker( + public DoFnInvoker newByteBuddyInvoker( + DoFn fn) { + return newByteBuddyInvoker(DoFnSignatures.INSTANCE.getOrParseSignature(fn.getClass()), fn); + } + + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + public DoFnInvoker newByteBuddyInvoker( DoFnSignature signature, DoFn fn) { - Preconditions.checkArgument( - signature.getFnClass().equals(fn.getClass()), + checkArgument( + signature.fnClass().equals(fn.getClass()), "Signature is for class %s, but fn is of class %s", - signature.getFnClass(), + signature.fnClass(), fn.getClass()); try { @SuppressWarnings("unchecked") @@ -106,26 +113,25 @@ static DoFnInvoker newByteBuddyInvoker( * Returns a generated constructor for a {@link DoFnInvoker} for the given {@link DoFn} class and * caches it. */ - private static synchronized Constructor getOrGenerateByteBuddyInvokerConstructor( + private synchronized Constructor getOrGenerateByteBuddyInvokerConstructor( DoFnSignature signature) { - Class fnClass = signature.getFnClass(); - Constructor constructor = BYTE_BUDDY_INVOKER_CONSTRUCTOR_CACHE.get(fnClass); - if (constructor != null) { - return constructor; - } - Class> invokerClass = generateInvokerClass(signature); - try { - constructor = invokerClass.getConstructor(fnClass); - } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) { - throw new RuntimeException(e); + Class fnClass = signature.fnClass(); + Constructor constructor = byteBuddyInvokerConstructorCache.get(fnClass); + if (constructor == null) { + Class> invokerClass = generateInvokerClass(signature); + try { + constructor = invokerClass.getConstructor(fnClass); + } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) { + throw new RuntimeException(e); + } + byteBuddyInvokerConstructorCache.put(fnClass, constructor); } - BYTE_BUDDY_INVOKER_CONSTRUCTOR_CACHE.put(fnClass, constructor); return constructor; } /** Generates a {@link DoFnInvoker} class for the given {@link DoFnSignature}. */ private static Class> generateInvokerClass(DoFnSignature signature) { - Class fnClass = signature.getFnClass(); + Class fnClass = signature.fnClass(); final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(fnClass); @@ -147,29 +153,28 @@ public String subclass(TypeDescription.Generic superClass) { .defineConstructor(Visibility.PUBLIC) .withParameter(fnClass) .intercept(new InvokerConstructor()) - // Delegate processElement(), startBundle() and finishBundle() to the fn. .method(ElementMatchers.named("invokeProcessElement")) - .intercept(new ProcessElementDelegation(signature.getProcessElement())) + .intercept(new ProcessElementDelegation(signature.processElement())) .method(ElementMatchers.named("invokeStartBundle")) .intercept( - signature.getStartBundle() == null + signature.startBundle() == null ? new NoopMethodImplementation() - : new BundleMethodDelegation(signature.getStartBundle())) + : new BundleMethodDelegation(signature.startBundle())) .method(ElementMatchers.named("invokeFinishBundle")) .intercept( - signature.getFinishBundle() == null + signature.finishBundle() == null ? new NoopMethodImplementation() - : new BundleMethodDelegation(signature.getFinishBundle())) + : new BundleMethodDelegation(signature.finishBundle())) .method(ElementMatchers.named("invokeSetup")) .intercept( - signature.getSetup() == null + signature.setup() == null ? new NoopMethodImplementation() - : new LifecycleMethodDelegation(signature.getSetup())) + : new LifecycleMethodDelegation(signature.setup())) .method(ElementMatchers.named("invokeTeardown")) .intercept( - signature.getTeardown() == null + signature.teardown() == null ? new NoopMethodImplementation() - : new LifecycleMethodDelegation(signature.getTeardown())); + : new LifecycleMethodDelegation(signature.teardown())); DynamicType.Unloaded unloaded = builder.make(); @@ -280,7 +285,8 @@ private static final class ProcessElementDelegation extends MethodDelegation { DoFn.ExtraContextFactory.class.getMethod("outputReceiver"))); EXTRA_CONTEXT_FACTORY_METHODS = Collections.unmodifiableMap(methods); } catch (Exception e) { - throw new RuntimeException(e); + throw new RuntimeException( + "Failed to locate an ExtraContextFactory method that was expected to exist", e); } } @@ -295,7 +301,7 @@ private ProcessElementDelegation(DoFnSignature.ProcessElementMethod signature) { protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { MethodDescription targetMethod = new MethodCall.MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod(signature.getTargetMethod())) + new MethodDescription.ForLoadedMethod(signature.targetMethod())) .resolve(instrumentedMethod); // Parameters of the wrapper invoker method: @@ -307,7 +313,7 @@ protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMet parameters.add(MethodVariableAccess.REFERENCE.loadOffset(1)); // Push the extra arguments in their actual order. StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadOffset(2); - for (DoFnSignature.ProcessElementMethod.Parameter param : signature.getExtraParameters()) { + for (DoFnSignature.ProcessElementMethod.Parameter param : signature.extraParameters()) { parameters.add( new StackManipulation.Compound( pushExtraContextFactory, @@ -340,7 +346,7 @@ private BundleMethodDelegation(@Nullable DoFnSignature.BundleMethod signature) { protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { MethodDescription targetMethod = new MethodCall.MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod(checkNotNull(signature).getTargetMethod())) + new MethodDescription.ForLoadedMethod(checkNotNull(signature).targetMethod())) .resolve(instrumentedMethod); return new StackManipulation.Compound( // Push the parameters @@ -353,8 +359,8 @@ protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMet } /** - * Implements {@link DoFnInvoker#invokeSetup} or {@link DoFnInvoker#invokeTeardown} by - * delegating respectively to the {@link Setup} and {@link Teardown} methods. + * Implements {@link DoFnInvoker#invokeSetup} or {@link DoFnInvoker#invokeTeardown} by delegating + * respectively to the {@link Setup} and {@link Teardown} methods. */ private static final class LifecycleMethodDelegation extends MethodDelegation { private final DoFnSignature.LifecycleMethod signature; @@ -367,7 +373,7 @@ private LifecycleMethodDelegation(@Nullable DoFnSignature.LifecycleMethod signat protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { MethodDescription targetMethod = new MethodCall.MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod(checkNotNull(signature).getTargetMethod())) + new MethodDescription.ForLoadedMethod(checkNotNull(signature).targetMethod())) .resolve(instrumentedMethod); return new StackManipulation.Compound( wrapWithUserCodeException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java new file mode 100644 index 000000000000..6730140ac309 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -0,0 +1,113 @@ +/* + * 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.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +import com.google.auto.value.AutoValue; + +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra + * context it requires, types of the input and output elements, etc. + * + *

See A new DoFn. + */ +@AutoValue +public abstract class DoFnSignature { + public abstract Class fnClass(); + + public abstract ProcessElementMethod processElement(); + + @Nullable + public abstract BundleMethod startBundle(); + + @Nullable + public abstract BundleMethod finishBundle(); + + @Nullable + public abstract LifecycleMethod setup(); + + @Nullable + public abstract LifecycleMethod teardown(); + + static DoFnSignature create( + Class fnClass, + ProcessElementMethod processElement, + @Nullable BundleMethod startBundle, + @Nullable BundleMethod finishBundle, + @Nullable LifecycleMethod setup, + @Nullable LifecycleMethod teardown) { + return new AutoValue_DoFnSignature( + fnClass, + processElement, + startBundle, + finishBundle, + setup, + teardown); + } + + /** Describes a {@link DoFn.ProcessElement} method. */ + @AutoValue + public abstract static class ProcessElementMethod { + enum Parameter { + BOUNDED_WINDOW, + INPUT_PROVIDER, + OUTPUT_RECEIVER + } + + public abstract Method targetMethod(); + + public abstract List extraParameters(); + + static ProcessElementMethod create(Method targetMethod, List extraParameters) { + return new AutoValue_DoFnSignature_ProcessElementMethod( + targetMethod, Collections.unmodifiableList(extraParameters)); + } + + /** @return true if the reflected {@link DoFn} uses a Single Window. */ + public boolean usesSingleWindow() { + return extraParameters().contains(Parameter.BOUNDED_WINDOW); + } + } + + /** Describes a {@link DoFn.StartBundle} or {@link DoFn.FinishBundle} method. */ + @AutoValue + public abstract static class BundleMethod { + public abstract Method targetMethod(); + + static BundleMethod create(Method targetMethod) { + return new AutoValue_DoFnSignature_BundleMethod(targetMethod); + } + } + + /** Describes a {@link DoFn.Setup} or {@link DoFn.Teardown} method. */ + @AutoValue + public abstract static class LifecycleMethod { + public abstract Method targetMethod(); + + static LifecycleMethod create(Method targetMethod) { + return new AutoValue_DoFnSignature_LifecycleMethod(targetMethod); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java similarity index 91% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index 20bf97ee8500..80b3b4f5b825 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -15,14 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.sdk.transforms.reflect; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.common.ReflectHelpers; -import com.google.common.base.Preconditions; +import com.google.common.annotations.VisibleForTesting; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; @@ -43,30 +45,23 @@ /** * Parses a {@link DoFn} and computes its {@link DoFnSignature}. See {@link #getOrParseSignature}. */ -public abstract class DoFnReflector { - private DoFnReflector() {} +public class DoFnSignatures { + public static final DoFnSignatures INSTANCE = new DoFnSignatures(); - private static final Map, DoFnSignature> SIGNATURE_CACHE = new LinkedHashMap<>(); + private DoFnSignatures() {} + + private final Map, DoFnSignature> signatureCache = new LinkedHashMap<>(); /** @return the {@link DoFnSignature} for the given {@link DoFn}. */ - static synchronized DoFnSignature getOrParseSignature( + public synchronized DoFnSignature getOrParseSignature( @SuppressWarnings("rawtypes") Class fn) { - DoFnSignature signature = SIGNATURE_CACHE.get(fn); - if (signature != null) { - return signature; + DoFnSignature signature = signatureCache.get(fn); + if (signature == null) { + signatureCache.put(fn, signature = parseSignature(fn)); } - - signature = parseSignature(fn); - SIGNATURE_CACHE.put(fn, signature); return signature; } - /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ - public static DoFnInvoker newByteBuddyInvoker( - DoFn fn) { - return DoFnInvokers.newByteBuddyInvoker(getOrParseSignature(fn.getClass()), fn); - } - /** Analyzes a given {@link DoFn} class and extracts its {@link DoFnSignature}. */ private static DoFnSignature parseSignature(Class fnClass) { TypeToken inputT = null; @@ -86,7 +81,7 @@ private static DoFnSignature parseSignature(Class fnClass) { inputT = TypeToken.of(args[0]); outputT = TypeToken.of(args[1]); } - Preconditions.checkNotNull(inputT, "Unable to determine input type from %s", fnClass); + checkNotNull(inputT, "Unable to determine input type from %s", fnClass); Method processElementMethod = findAnnotatedMethod(DoFn.ProcessElement.class, fnClass, true); Method startBundleMethod = findAnnotatedMethod(DoFn.StartBundle.class, fnClass, false); @@ -94,10 +89,8 @@ private static DoFnSignature parseSignature(Class fnClass) { Method setupMethod = findAnnotatedMethod(DoFn.Setup.class, fnClass, false); Method teardownMethod = findAnnotatedMethod(DoFn.Teardown.class, fnClass, false); - return new DoFnSignature( + return DoFnSignature.create( fnClass, - inputT, - outputT, analyzeProcessElementMethod(fnToken, processElementMethod, inputT, outputT), (startBundleMethod == null) ? null @@ -146,6 +139,7 @@ private static TypeToken> outputReceiverT new TypeParameter() {}, inputT); } + @VisibleForTesting static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( TypeToken fnClass, Method m, TypeToken inputT, TypeToken outputT) { checkArgument( @@ -213,9 +207,10 @@ static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( } } - return new DoFnSignature.ProcessElementMethod(m, extraParameters); + return DoFnSignature.ProcessElementMethod.create(m, extraParameters); } + @VisibleForTesting static DoFnSignature.BundleMethod analyzeBundleMethod( TypeToken fnToken, Method m, TypeToken inputT, TypeToken outputT) { checkArgument( @@ -238,15 +233,15 @@ static DoFnSignature.BundleMethod analyzeBundleMethod( formatType(contextToken), formatType(expectedContextToken)); - return new DoFnSignature.BundleMethod(m); + return DoFnSignature.BundleMethod.create(m); } - static DoFnSignature.LifecycleMethod analyzeLifecycleMethod(Method m) { + private static DoFnSignature.LifecycleMethod analyzeLifecycleMethod(Method m) { checkArgument( void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); checkArgument( m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); - return new DoFnSignature.LifecycleMethod(m); + return DoFnSignature.LifecycleMethod.create(m); } private static Collection declaredMethodsWithAnnotation( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java new file mode 100644 index 000000000000..1c920a529e99 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * Defines the {@link org.apache.beam.sdk.transforms.join.CoGroupByKey} transform + * for joining multiple PCollections. + */ +package org.apache.beam.sdk.transforms.reflect; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java similarity index 88% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 22d1d2794876..7e756e245f91 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.sdk.transforms.reflect; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.transforms.dofnreflector.DoFnInvokersTestHelper; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; + import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -49,7 +50,7 @@ public Invocations(String name) { @Rule public ExpectedException thrown = ExpectedException.none(); - @Mock private DoFn.ProcessContext mockContext; + @Mock private DoFn.ProcessContext mockContext; @Mock private BoundedWindow mockWindow; @Mock private DoFn.InputProvider mockInputProvider; @Mock private DoFn.OutputReceiver mockOutputReceiver; @@ -86,7 +87,9 @@ private void checkInvokeProcessElementWorks(DoFn fn, Invocations "Should not yet have called processElement on " + invocation.name, invocation.wasProcessElementInvoked); } - DoFnReflector.newByteBuddyInvoker(fn).invokeProcessElement(mockContext, extraContextFactory); + DoFnInvokers.INSTANCE + .newByteBuddyInvoker(fn) + .invokeProcessElement(mockContext, extraContextFactory); for (Invocations invocation : invocations) { assertTrue( "Should have called processElement on " + invocation.name, @@ -102,7 +105,7 @@ private void checkInvokeStartBundleWorks(DoFn fn, Invocations... "Should not yet have called startBundle on " + invocation.name, invocation.wasStartBundleInvoked); } - DoFnReflector.newByteBuddyInvoker(fn).invokeStartBundle(mockContext); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeStartBundle(mockContext); for (Invocations invocation : invocations) { assertTrue( "Should have called startBundle on " + invocation.name, invocation.wasStartBundleInvoked); @@ -117,7 +120,7 @@ private void checkInvokeFinishBundleWorks(DoFn fn, Invocations.. "Should not yet have called finishBundle on " + invocation.name, invocation.wasFinishBundleInvoked); } - DoFnReflector.newByteBuddyInvoker(fn).invokeFinishBundle(mockContext); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeFinishBundle(mockContext); for (Invocations invocation : invocations) { assertTrue( "Should have called finishBundle on " + invocation.name, @@ -132,7 +135,7 @@ private void checkInvokeSetupWorks(DoFn fn, Invocations... invoc assertFalse( "Should not yet have called setup on " + invocation.name, invocation.wasSetupInvoked); } - DoFnReflector.newByteBuddyInvoker(fn).invokeSetup(); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeSetup(); for (Invocations invocation : invocations) { assertTrue("Should have called setup on " + invocation.name, invocation.wasSetupInvoked); } @@ -146,7 +149,7 @@ private void checkInvokeTeardownWorks(DoFn fn, Invocations... in "Should not yet have called teardown on " + invocation.name, invocation.wasTeardownInvoked); } - DoFnReflector.newByteBuddyInvoker(fn).invokeTeardown(); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeTeardown(); for (Invocations invocation : invocations) { assertTrue( "Should have called teardown on " + invocation.name, invocation.wasTeardownInvoked); @@ -166,20 +169,23 @@ public void processElement(ProcessContext c) throws Exception { }; assertFalse( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, invocations); } @Test public void testDoFnInvokersReused() throws Exception { - // Ensures that we don't create a new Invoker class for every instance of the OldDoFn. + // Ensures that we don't create a new Invoker class for every instance of the DoFn. IdentityParent fn1 = new IdentityParent(); IdentityParent fn2 = new IdentityParent(); assertSame( "Invoker classes should only be generated once for each type", - DoFnReflector.newByteBuddyInvoker(fn1).getClass(), - DoFnReflector.newByteBuddyInvoker(fn2).getClass()); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn1).getClass(), + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn2).getClass()); } interface InterfaceWithProcessElement { @@ -205,7 +211,10 @@ public void processElement(DoFn.ProcessContext c) { public void testDoFnWithProcessElementInterface() throws Exception { IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); assertFalse( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, fn.invocations); } @@ -235,7 +244,10 @@ public void process(DoFn.ProcessContext c) { public void testDoFnWithMethodInSuperclass() throws Exception { IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); assertFalse( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, fn.parentInvocations); } @@ -243,7 +255,10 @@ public void testDoFnWithMethodInSuperclass() throws Exception { public void testDoFnWithMethodInSubclass() throws Exception { IdentityChildWithOverride fn = new IdentityChildWithOverride(); assertFalse( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, fn.parentInvocations, fn.childInvocations); } @@ -261,7 +276,10 @@ public void processElement(ProcessContext c, BoundedWindow w) throws Exception { }; assertTrue( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, invocations); } @@ -280,7 +298,10 @@ public void processElement(ProcessContext c, OutputReceiver o) throws Ex }; assertFalse( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, invocations); } @@ -299,7 +320,10 @@ public void processElement(ProcessContext c, InputProvider i) throws Exc }; assertFalse( - DoFnReflector.getOrParseSignature(fn.getClass()).getProcessElement().usesSingleWindow()); + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); checkInvokeProcessElementWorks(fn, invocations); } @@ -433,7 +457,7 @@ public void processElement(@SuppressWarnings("unused") ProcessContext c) { thrown.expect(UserCodeException.class); thrown.expectMessage("bogus"); - DoFnReflector.newByteBuddyInvoker(fn).invokeProcessElement(null, null); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeProcessElement(null, null); } @Test @@ -451,7 +475,7 @@ public void processElement(@SuppressWarnings("unused") ProcessContext c) {} thrown.expect(UserCodeException.class); thrown.expectMessage("bogus"); - DoFnReflector.newByteBuddyInvoker(fn).invokeStartBundle(null); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeStartBundle(null); } @Test @@ -469,6 +493,6 @@ public void processElement(@SuppressWarnings("unused") ProcessContext c) {} thrown.expect(UserCodeException.class); thrown.expectMessage("bogus"); - DoFnReflector.newByteBuddyInvoker(fn).invokeFinishBundle(null); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeFinishBundle(null); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnInvokersTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java similarity index 95% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnInvokersTestHelper.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java index d3ad3e521f85..7bfdddc05226 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnInvokersTestHelper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java @@ -15,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms.dofnreflector; +package org.apache.beam.sdk.transforms.reflect; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnInvokersTest; -import org.apache.beam.sdk.transforms.DoFnInvokersTest.Invocations; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokersTest.Invocations; /** * Test helper for {@link DoFnInvokersTest}, which needs to test package-private access diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java similarity index 90% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index 0d8cfb0ca404..1a26df22cc34 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -15,7 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.sdk.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; import com.google.common.reflect.TypeToken; @@ -28,9 +30,9 @@ import java.lang.reflect.Method; import java.util.List; -/** Tests for {@link DoFnReflector}. */ +/** Tests for {@link DoFnSignatures}. */ @RunWith(JUnit4.class) -public class DoFnReflectorTest { +public class DoFnSignaturesTest { @Rule public ExpectedException thrown = ExpectedException.none(); @@ -46,7 +48,7 @@ public void testMissingProcessContext() throws Exception { getClass().getName() + "#missingProcessContext() must take a ProcessContext<> as its first argument"); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), getClass().getDeclaredMethod("missingProcessContext"), TypeToken.of(Integer.class), @@ -63,7 +65,7 @@ public void testBadProcessContextType() throws Exception { getClass().getName() + "#badProcessContext(String) must take a ProcessContext<> as its first argument"); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), getClass().getDeclaredMethod("badProcessContext", String.class), TypeToken.of(Integer.class), @@ -80,7 +82,7 @@ public void testBadExtraContext() throws Exception { getClass().getName() + "#badExtraContext(Context, int) must have a single argument of type Context"); - DoFnReflector.analyzeBundleMethod( + DoFnSignatures.analyzeBundleMethod( TypeToken.of(FakeDoFn.class), getClass().getDeclaredMethod("badExtraContext", DoFn.Context.class, int.class), TypeToken.of(Integer.class), @@ -99,7 +101,7 @@ public void testBadExtraProcessContextType() throws Exception { + "#badExtraProcessContext(ProcessContext, Integer)" + ". Should be one of [BoundedWindow]"); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), getClass() .getDeclaredMethod("badExtraProcessContext", DoFn.ProcessContext.class, Integer.class), @@ -117,7 +119,7 @@ public void testBadReturnType() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage(getClass().getName() + "#badReturnType() must have a void return type"); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), getClass().getDeclaredMethod("badReturnType"), TypeToken.of(Integer.class), @@ -139,7 +141,7 @@ public void testGoodConcreteTypes() throws Exception { DoFn.ProcessContext.class, DoFn.InputProvider.class, DoFn.OutputReceiver.class); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), method, TypeToken.of(Integer.class), @@ -157,7 +159,7 @@ public void goodTypeVariables( @Test public void testGoodTypeVariables() throws Exception { - DoFnReflector.getOrParseSignature(GoodTypeVariables.class); + DoFnSignatures.INSTANCE.getOrParseSignature(GoodTypeVariables.class); } private static class IdentityFn extends DoFn { @@ -172,7 +174,7 @@ private static class IdentityListFn extends IdentityFn> {} @Test public void testIdentityFnApplied() throws Exception { - DoFnReflector.getOrParseSignature(new IdentityFn() {}.getClass()); + DoFnSignatures.INSTANCE.getOrParseSignature(new IdentityFn() {}.getClass()); } @SuppressWarnings("unused") @@ -200,7 +202,7 @@ public void testBadGenericsTwoArgs() throws Exception { + "OutputReceiver, should be " + "OutputReceiver"); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), method, TypeToken.of(Integer.class), @@ -231,7 +233,7 @@ public void testBadGenericWildCards() throws Exception { + "OutputReceiver, should be " + "OutputReceiver"); - DoFnReflector.analyzeProcessElementMethod( + DoFnSignatures.analyzeProcessElementMethod( TypeToken.of(FakeDoFn.class), method, TypeToken.of(Integer.class), @@ -257,7 +259,7 @@ public void testBadTypeVariables() throws Exception { + "OutputReceiver, should be " + "OutputReceiver"); - DoFnReflector.getOrParseSignature(BadTypeVariables.class); + DoFnSignatures.INSTANCE.getOrParseSignature(BadTypeVariables.class); } @Test @@ -265,7 +267,7 @@ public void testNoProcessElement() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("No method annotated with @ProcessElement found"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature(new DoFn() {}.getClass()); + DoFnSignatures.INSTANCE.getOrParseSignature(new DoFn() {}.getClass()); } @Test @@ -275,7 +277,7 @@ public void testMultipleProcessElement() throws Exception { thrown.expectMessage("foo()"); thrown.expectMessage("bar()"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature( + DoFnSignatures.INSTANCE.getOrParseSignature( new DoFn() { @ProcessElement public void foo() {} @@ -292,7 +294,7 @@ public void testMultipleStartBundleElement() throws Exception { thrown.expectMessage("bar()"); thrown.expectMessage("baz()"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature( + DoFnSignatures.INSTANCE.getOrParseSignature( new DoFn() { @ProcessElement public void foo() {} @@ -312,7 +314,7 @@ public void testMultipleFinishBundleMethods() throws Exception { thrown.expectMessage("bar(Context)"); thrown.expectMessage("baz(Context)"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature( + DoFnSignatures.INSTANCE.getOrParseSignature( new DoFn() { @ProcessElement public void foo(ProcessContext context) {} @@ -330,7 +332,7 @@ public void testPrivateProcessElement() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("process() must be public"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature( + DoFnSignatures.INSTANCE.getOrParseSignature( new DoFn() { @ProcessElement private void process() {} @@ -342,7 +344,7 @@ public void testPrivateStartBundle() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("startBundle() must be public"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature( + DoFnSignatures.INSTANCE.getOrParseSignature( new DoFn() { @ProcessElement public void processElement() {} @@ -357,7 +359,7 @@ public void testPrivateFinishBundle() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("finishBundle() must be public"); thrown.expectMessage(getClass().getName() + "$"); - DoFnReflector.getOrParseSignature( + DoFnSignatures.INSTANCE.getOrParseSignature( new DoFn() { @ProcessElement public void processElement() {} diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java similarity index 85% rename from sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java rename to sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java index 8d9c344bb858..a574ed83dc42 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java @@ -23,15 +23,14 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; import org.apache.beam.sdk.transforms.DoFnAdapters; -import org.apache.beam.sdk.transforms.DoFnInvoker; -import org.apache.beam.sdk.transforms.DoFnReflector; import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Instant; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; @@ -41,23 +40,22 @@ import org.openjdk.jmh.annotations.Warmup; /** - * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically - * for measuring the overhead of {@link DoFnReflector}. + * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically for measuring the + * overhead of {@link DoFnInvokers}. */ @State(Scope.Benchmark) @Fork(1) @Warmup(iterations = 5) -public class DoFnReflectorBenchmark { +public class DoFnInvokersBenchmark { private static final String ELEMENT = "some string to use for testing"; private OldDoFn oldDoFn = new UpperCaseOldDoFn(); private DoFn doFn = new UpperCaseDoFn(); - private StubOldDoFnProcessContext stubOldDoFnContext = new StubOldDoFnProcessContext(oldDoFn, - ELEMENT); - private StubDoFnProcessContext stubDoFnContext = - new StubDoFnProcessContext(doFn, ELEMENT); + private StubOldDoFnProcessContext stubOldDoFnContext = + new StubOldDoFnProcessContext(oldDoFn, ELEMENT); + private StubDoFnProcessContext stubDoFnContext = new StubDoFnProcessContext(doFn, ELEMENT); private ExtraContextFactory extraContextFactory = new DoFn.FakeExtraContextFactory<>(); @@ -68,7 +66,7 @@ public class DoFnReflectorBenchmark { @Setup public void setUp() { adaptedDoFnWithContext = DoFnAdapters.toOldDoFn(doFn); - invoker = DoFnReflector.newByteBuddyInvoker(doFn); + invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(doFn); } @Benchmark @@ -161,24 +159,21 @@ public void outputWithTimestamp(String output, Instant timestamp) { } @Override - public void sideOutput(TupleTag tag, T output) { - } + public void sideOutput(TupleTag tag, T output) {} @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - } + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) {} @Override - protected Aggregator - createAggregatorInternal(String name, CombineFn combiner) { + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { return null; } } - private static class StubDoFnProcessContext - extends DoFn.ProcessContext { + private static class StubDoFnProcessContext extends DoFn.ProcessContext { private final String element; - private String output; + private String output; public StubDoFnProcessContext(DoFn fn, String element) { fn.super(); @@ -221,11 +216,9 @@ public void outputWithTimestamp(String output, Instant timestamp) { } @Override - public void sideOutput(TupleTag tag, T output) { - } + public void sideOutput(TupleTag tag, T output) {} @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - } + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) {} } } From 6f71f2959fb0a5485dc77bb503760385ce247ca0 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 17 Aug 2016 14:05:45 -0700 Subject: [PATCH 3/4] Some more javadocs --- .../main/java/org/apache/beam/sdk/transforms/DoFn.java | 4 ++-- .../java/org/apache/beam/sdk/transforms/DoFnAdapters.java | 8 +++++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 81d34c1eddef..2348783f9f61 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -312,12 +312,12 @@ public interface ExtraContextFactory { OutputReceiver outputReceiver(); } - /** A placeholder for testing purposes. */ + /** A placeholder for testing handling of output types during {@link DoFn} reflection. */ public interface OutputReceiver { void output(T output); } - /** A placeholder for testing purposes. */ + /** A placeholder for testing handling of input types during {@link DoFn} reflection. */ public interface InputProvider { T get(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java index 6bac4274216a..71a148ffafe9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -34,7 +34,13 @@ import java.io.IOException; -/** Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. */ +/** + * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. + * + * @deprecated This class will go away when we start running {@link DoFn}'s directly (using + * {@link DoFnInvoker}) rather than via {@link OldDoFn}. + */ +@Deprecated public class DoFnAdapters { /** Should not be instantiated. */ private DoFnAdapters() {} From 65eaa9c22d42bfaaaa9ab2063454aee9ed40e273 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 17 Aug 2016 15:40:42 -0700 Subject: [PATCH 4/4] javadoc fix --- .../org/apache/beam/sdk/transforms/reflect/package-info.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java index 1c920a529e99..4df5209e6b39 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java @@ -16,7 +16,8 @@ * limitations under the License. */ /** - * Defines the {@link org.apache.beam.sdk.transforms.join.CoGroupByKey} transform - * for joining multiple PCollections. + * Defines reflection-based utilities for analyzing {@link org.apache.beam.sdk.transforms.DoFn}'s + * and creating {@link org.apache.beam.sdk.transforms.reflect.DoFnSignature}'s and + * {@link org.apache.beam.sdk.transforms.reflect.DoFnInvoker}'s from them. */ package org.apache.beam.sdk.transforms.reflect;