@Test public void materializedColShouldBeWritten() throws Exception { File textFile = tmpDir.copyResourceFile("shakes.txt"); Pipeline pipeline = new MRPipeline(MRPipelineIT.class, tmpDir.getDefaultConfiguration()); PCollection<String> genericCollection = pipeline.readTextFile(textFile.getAbsolutePath()); pipeline.run(); PCollection<String> filter = genericCollection.filter("Filtering data", FilterFns.<String>ACCEPT_ALL()); filter.materialize(); pipeline.run(); File file = tmpDir.getFile("output.txt"); Target outFile = To.textFile(file.getAbsolutePath()); PCollection<String> write = filter.write(outFile); write.materialize(); pipeline.run(); }
@Test public void testMapsideJoin_RightSideIsEmpty() throws IOException { MRPipeline pipeline = new MRPipeline(MapsideJoinStrategyIT.class, tmpDir.getDefaultConfiguration()); PTable<Integer, String> customerTable = readTable(pipeline, "customers.txt"); PTable<Integer, String> orderTable = readTable(pipeline, "orders.txt"); PTable<Integer, String> filteredOrderTable = orderTable.parallelDo( FilterFns.<Pair<Integer, String>>REJECT_ALL(), orderTable.getPTableType()); JoinStrategy<Integer, String, String> mapsideJoin = new MapsideJoinStrategy<Integer, String, String>(); PTable<Integer, Pair<String, String>> joined = mapsideJoin.join(customerTable, filteredOrderTable, JoinType.INNER_JOIN); List<Pair<Integer, Pair<String, String>>> materializedJoin = Lists.newArrayList(joined.materialize()); assertTrue(materializedJoin.isEmpty()); }