PCollection.createPrimitiveOutputInternal( pipeline, windowingStrategy, isBounded, coders.get(outputTag)) .setTypeDescriptor((TypeDescriptor) outputTag.getTypeDescriptor());
public static <OutputT> PCollectionTuple createPrimitiveOutputFor( PCollection<?> input, DoFn<?, OutputT> fn, TupleTag<OutputT> mainOutputTag, TupleTagList additionalOutputTags, Map<TupleTag<?>, Coder<?>> outputTagsToCoders, WindowingStrategy<?, ?> windowingStrategy) { DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), outputTagsToCoders, windowingStrategy, input.isBounded().and(signature.isBoundedPerElement())); // Set output type descriptor similarly to how ParDo.MultiOutput does it. outputs.get(mainOutputTag).setTypeDescriptor(fn.getOutputTypeDescriptor()); return outputs; }
@Test public void testWritingDataFailInvalidValueType() { conf.set(HadoopFormatIO.OUTPUT_DIR, tmpFolder.getRoot().getAbsolutePath()); List<KV<Text, Text>> data = new ArrayList<>(); data.add(KV.of(new Text("key"), new Text("value"))); TypeDescriptor<Text> textTypeDescriptor = new TypeDescriptor<Text>() {}; PCollection<KV<Text, Text>> input = p.apply(Create.of(data)) .setTypeDescriptor(TypeDescriptors.kvs(textTypeDescriptor, textTypeDescriptor)); thrown.expect(Pipeline.PipelineExecutionException.class); thrown.expectMessage(Text.class.getName()); input.apply( "Write", HadoopFormatIO.<Text, Text>write() .withConfiguration(conf) .withPartitioning() .withExternalSynchronization(new HDFSSynchronization(getLocksDirPath()))); p.run().waitUntilFinish(); }
@Override public PCollection<KV<Integer, KV<KeyT, ValueT>>> expand(PCollection<KV<KeyT, ValueT>> input) { return input .apply( "AssignTask", ParDo.of(new AssignTaskFn<KeyT, ValueT>(configView)).withSideInputs(configView)) .setTypeDescriptor( TypeDescriptors.kvs(TypeDescriptors.integers(), input.getTypeDescriptor())) .apply("GroupByTaskId", GroupByKey.create()) .apply("FlattenGroupedTasks", ParDo.of(new FlattenGroupedTasks<>())); } }
@Test public void testWritingData() throws IOException { conf.set(HadoopFormatIO.OUTPUT_DIR, tmpFolder.getRoot().getAbsolutePath()); List<KV<Text, Employee>> data = TestEmployeeDataSet.getEmployeeData(); PCollection<KV<Text, Employee>> input = p.apply(Create.of(data)) .setTypeDescriptor( TypeDescriptors.kvs( new TypeDescriptor<Text>() {}, new TypeDescriptor<Employee>() {})); input.apply( "Write", HadoopFormatIO.<Text, Employee>write() .withConfiguration(conf) .withPartitioning() .withExternalSynchronization(new HDFSSynchronization(getLocksDirPath()))); p.run(); List<KV<Text, Employee>> writtenOutput = EmployeeOutputFormat.getWrittenOutput(); assertEquals(data.size(), writtenOutput.size()); assertTrue(data.containsAll(writtenOutput)); assertTrue(writtenOutput.containsAll(data)); Mockito.verify(EmployeeOutputFormat.getOutputCommitter()).commitJob(Mockito.any()); Mockito.verify(EmployeeOutputFormat.getOutputCommitter(), Mockito.times(REDUCERS_COUNT)) .commitTask(Mockito.any()); }
private void executeBatchTest(HadoopFormatIO.Write<Text, LongWritable> write, String outputDir) { pipeline .apply(Create.of(SENTENCES)) .apply(ParDo.of(new ConvertToLowerCaseFn())) .apply(new WordCount.CountWords()) .apply( "ConvertToHadoopFormat", ParDo.of(new ConvertToHadoopFormatFn<>(KV_STR_INT_2_TXT_LONGWRITABLE))) .setTypeDescriptor( TypeDescriptors.kvs( new TypeDescriptor<Text>() {}, new TypeDescriptor<LongWritable>() {})) .apply(write); pipeline.run(); Map<String, Long> results = loadWrittenDataAsMap(outputDir); MatcherAssert.assertThat(results.entrySet(), equalTo(computeWordCounts(SENTENCES).entrySet())); }
@Test public void testWritingDataFailInvalidKeyType() { conf.set(HadoopFormatIO.OUTPUT_DIR, tmpFolder.getRoot().getAbsolutePath()); List<KV<String, Employee>> data = new ArrayList<>(); data.add(KV.of("key", new Employee("name", "address"))); PCollection<KV<String, Employee>> input = p.apply("CreateData", Create.of(data)) .setTypeDescriptor( TypeDescriptors.kvs( new TypeDescriptor<String>() {}, new TypeDescriptor<Employee>() {})); thrown.expect(Pipeline.PipelineExecutionException.class); thrown.expectMessage(String.class.getName()); input.apply( "Write", HadoopFormatIO.<String, Employee>write() .withConfiguration(conf) .withPartitioning() .withExternalSynchronization(new HDFSSynchronization(getLocksDirPath()))); p.run().waitUntilFinish(); }
outputs.get(mainOutputTag).setTypeDescriptor(getFn().getOutputTypeDescriptor());
private void runValidationPipeline(Configuration configuration) { p.apply(Create.of(TestEmployeeDataSet.getEmployeeData())) .setTypeDescriptor( TypeDescriptors.kvs(new TypeDescriptor<Text>() {}, new TypeDescriptor<Employee>() {})) .apply( "Write", HadoopFormatIO.<Text, Employee>write() .withConfiguration(configuration) .withPartitioning() .withExternalSynchronization(new HDFSSynchronization(getLocksDirPath()))); }
ParDo.of(new WriteFn<KeyT, ValueT>(configView, externalSynchronization)) .withSideInputs(configView)) .setTypeDescriptor(TypeDescriptors.integers()) .apply( "CollectWriteTasks", Combine.globally(new IterableCombinerFn<>(TypeDescriptors.integers())) .withoutDefaults()) .setTypeDescriptor(iterableIntType);
"ConvertToHadoopFormat", ParDo.of(new ConvertToHadoopFormatFn<>(KV_STR_INT_2_TXT_LONGWRITABLE))) .setTypeDescriptor( TypeDescriptors.kvs( new TypeDescriptor<Text>() {}, new TypeDescriptor<LongWritable>() {}));
/** * This test validates functionality of {@link * HadoopFormatIO.Write.Builder#withConfiguration(Configuration) withConfiguration(Configuration)} * function when Hadoop OutputFormat class is not provided by the user in configuration. */ @Test public void testWriteValidationFailsMissingOutputFormatInConf() { Configuration configuration = new Configuration(); configuration.setClass(HadoopFormatIO.OUTPUT_KEY_CLASS, Text.class, Object.class); configuration.setClass(HadoopFormatIO.OUTPUT_VALUE_CLASS, Employee.class, Object.class); HadoopFormatIO.Write<Text, Employee> writeWithWrongConfig = HadoopFormatIO.<Text, Employee>write() .withConfiguration(configuration) .withPartitioning() .withExternalSynchronization(new HDFSSynchronization(getLocksDirPath())); p.apply(Create.of(TestEmployeeDataSet.getEmployeeData())) .setTypeDescriptor( TypeDescriptors.kvs(new TypeDescriptor<Text>() {}, new TypeDescriptor<Employee>() {})) .apply("Write", writeWithWrongConfig); thrown.expect(Pipeline.PipelineExecutionException.class); thrown.expectMessage("Configuration must contain \"mapreduce.job.outputformat.class\""); p.run().waitUntilFinish(); }