@Override public OutputT expand(InputT input) { OutputT res = delegate().expand(input); if (res instanceof PCollection) { PCollection pc = (PCollection) res; try { pc.setCoder(delegate().getDefaultOutputCoder(input, pc)); } catch (CannotProvideCoderException e) { // Let coder inference happen later. } } return res; }
@Test public void applyDelegates() { @SuppressWarnings("unchecked") PCollection<Integer> collection = mock(PCollection.class); @SuppressWarnings("unchecked") PCollection<String> output = mock(PCollection.class); when(delegate.expand(collection)).thenReturn(output); PCollection<String> result = forwarding.expand(collection); assertThat(result, equalTo(output)); }
/** * Applies a {@link PTransform} to the given {@link PInput}. * * @see Pipeline#apply */ private <InputT extends PInput, OutputT extends POutput> OutputT applyInternal( String name, InputT input, PTransform<? super InputT, OutputT> transform) { String namePrefix = transforms.getCurrent().getFullName(); String uniqueName = uniquifyInternal(namePrefix, name); final String builtName = buildName(namePrefix, name); instancePerName.put(builtName, transform); LOG.debug("Adding {} to {}", transform, this); transforms.pushNode(uniqueName, input, transform); try { transforms.finishSpecifyingInput(); OutputT output = transform.expand(input); transforms.setOutput(output); return output; } finally { transforms.popNode(); } }
@Test public void getDefaultOutputCoderDelegates() throws Exception { @SuppressWarnings("unchecked") PCollection<Integer> input = PCollection.createPrimitiveOutputInternal( null /* pipeline */, WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED, null /* coder */); @SuppressWarnings("unchecked") PCollection<String> output = PCollection.createPrimitiveOutputInternal( null /* pipeline */, WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED, null /* coder */); @SuppressWarnings("unchecked") Coder<String> outputCoder = mock(Coder.class); when(delegate.expand(input)).thenReturn(output); when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder); assertThat(forwarding.expand(input).getCoder(), equalTo(outputCoder)); }
private < InputT extends PInput, OutputT extends POutput, TransformT extends PTransform<? super InputT, OutputT>> void applyReplacement( Node original, PTransformOverrideFactory<InputT, OutputT, TransformT> replacementFactory) { PTransformReplacement<InputT, OutputT> replacement = replacementFactory.getReplacementTransform( (AppliedPTransform<InputT, OutputT, TransformT>) original.toAppliedPTransform(this)); if (replacement.getTransform() == original.getTransform()) { return; } InputT originalInput = replacement.getInput(); LOG.debug("Replacing {} with {}", original, replacement); transforms.replaceNode(original, originalInput, replacement.getTransform()); try { OutputT newOutput = replacement.getTransform().expand(originalInput); Map<PValue, ReplacementOutput> originalToReplacement = replacementFactory.mapOutputs(original.getOutputs(), newOutput); // Ensure the internal TransformHierarchy data structures are consistent. transforms.setOutput(newOutput); transforms.replaceOutputs(originalToReplacement); } finally { transforms.popNode(); } }