@Test public void testCreateNormalParDoFn() throws Exception { String stringState = "some state"; long longState = 42L; TestDoFn fn = new TestDoFn(stringState, longState); String serializedFn = StringUtils.byteArrayToJsonString( SerializableUtils.serializeToByteArray( new DoFnInfo(fn, WindowingStrategy.globalDefault()))); CloudObject cloudUserFn = CloudObject.forClassName("DoFn"); addString(cloudUserFn, "serialized_fn", serializedFn); String tag = "output"; MultiOutputInfo multiOutputInfo = new MultiOutputInfo(); multiOutputInfo.setTag(tag); List<MultiOutputInfo> multiOutputInfos = Arrays.asList(multiOutputInfo); PipelineOptions options = PipelineOptionsFactory.create(); DataflowExecutionContext context = BatchModeExecutionContext.fromOptions(options); CounterSet counters = new CounterSet(); StateSampler stateSampler = new StateSampler("test", counters.getAddCounterMutator()); ParDoFn parDoFn = factory.create( options, cloudUserFn, "name", "transformName", null, multiOutputInfos, 1, context, counters.getAddCounterMutator(), stateSampler); // Test that the factory created the correct class assertThat(parDoFn, instanceOf(NormalParDoFn.class)); // Test that the DoFnInfo reflects the one passed in NormalParDoFn normalParDoFn = (NormalParDoFn) parDoFn; DoFnInfo doFnInfo = normalParDoFn.getDoFnInfo(); DoFn actualDoFn = doFnInfo.getDoFn(); assertThat(actualDoFn, instanceOf(TestDoFn.class)); assertThat(doFnInfo.getWindowingStrategy().getWindowFn(), instanceOf(GlobalWindows.class)); assertThat( doFnInfo.getWindowingStrategy().getTrigger().getSpec(), instanceOf(DefaultTrigger.class)); // Test that the deserialized user DoFn is as expected TestDoFn actualTestDoFn = (TestDoFn) actualDoFn; assertEquals(stringState, actualTestDoFn.stringState); assertEquals(longState, actualTestDoFn.longState); assertEquals(context, normalParDoFn.getExecutionContext()); }
@Test public void testCreateUnknownParDoFn() throws Exception { CloudObject cloudUserFn = CloudObject.forClassName("UnknownKindOfDoFn"); try { CounterSet counters = new CounterSet(); StateSampler stateSampler = new StateSampler("test", counters.getAddCounterMutator()); factory.create( PipelineOptionsFactory.create(), cloudUserFn, "name", "transformName", null, null, 1, BatchModeExecutionContext.fromOptions(PipelineOptionsFactory.create()), counters.getAddCounterMutator(), stateSampler); fail("should have thrown an exception"); } catch (Exception exn) { assertThat(exn.toString(), Matchers.containsString("No known ParDoFnFactory")); } }
/** * Handle the provided {@link InProcessTransformResult}, produced after evaluating the provided * {@link CommittedBundle} (potentially null, if the result of a root {@link PTransform}). * * <p>The result is the output of running the transform contained in the {@link * InProcessTransformResult} on the contents of the provided bundle. * * @param completedBundle the bundle that was processed to produce the result. Potentially {@code * null} if the transform that produced the result is a root transform * @param completedTimers the timers that were delivered to produce the {@code completedBundle}, * or an empty iterable if no timers were delivered * @param result the result of evaluating the input bundle * @return the committed bundles contained within the handled {@code result} */ public CommittedResult handleResult( @Nullable CommittedBundle<?> completedBundle, Iterable<TimerData> completedTimers, InProcessTransformResult result) { Iterable<? extends CommittedBundle<?>> committedBundles = commitBundles(result.getOutputBundles()); // Update watermarks and timers EnumSet<OutputType> outputTypes = EnumSet.copyOf(result.getOutputTypes()); if (Iterables.isEmpty(committedBundles)) { outputTypes.remove(OutputType.BUNDLE); } else { outputTypes.add(OutputType.BUNDLE); } CommittedResult committedResult = CommittedResult.create( result, completedBundle == null ? null : completedBundle.withElements((Iterable) result.getUnprocessedElements()), committedBundles, outputTypes); watermarkManager.updateWatermarks( completedBundle, result.getTimerUpdate().withCompletedTimers(completedTimers), committedResult, result.getWatermarkHold()); // Update counters if (result.getCounters() != null) { mergedCounters.merge(result.getCounters()); } // Update state internals CopyOnAccessInMemoryStateInternals<?> theirState = result.getState(); if (theirState != null) { CopyOnAccessInMemoryStateInternals<?> committedState = theirState.commit(); StepAndKey stepAndKey = StepAndKey.of( result.getTransform(), completedBundle == null ? null : completedBundle.getKey()); if (!committedState.isEmpty()) { applicationStateInternals.put(stepAndKey, committedState); } else { applicationStateInternals.remove(stepAndKey); } } return committedResult; }