write = AvroIO.write(GenericClass.class) .to(policy) .withTempDirectory( StaticValueProvider.of( FileSystems.matchNewResource(baseDir.toString(), true))) .withWindowedWrites() .withNumShards(2) .withOutputFilenames(); break;
@SuppressWarnings("deprecation") // using AvroCoder#createDatumReader for tests. private void runTestWrite(String[] expectedElements, int numShards) throws IOException { File baseOutputFile = new File(tmpFolder.getRoot(), "prefix"); String outputFilePrefix = baseOutputFile.getAbsolutePath(); AvroIO.Write<String> write = AvroIO.write(String.class).to(outputFilePrefix).withSuffix(".avro"); if (numShards > 1) { write = write.withNumShards(numShards); } else { write = write.withoutSharding(); } writePipeline.apply(Create.of(ImmutableList.copyOf(expectedElements))).apply(write); writePipeline.run(); String shardNameTemplate = firstNonNull( write.inner.getShardTemplate(), DefaultFilenamePolicy.DEFAULT_UNWINDOWED_SHARD_TEMPLATE); assertTestOutputs(expectedElements, numShards, outputFilePrefix, shardNameTemplate); }
@Test @SuppressWarnings("unchecked") @Category(NeedsRunner.class) public void testCompressedWriteAndReadASingleFile() throws Throwable { List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); writePipeline .apply(Create.of(values)) .apply( AvroIO.write(GenericClass.class) .to(outputFile.getAbsolutePath()) .withoutSharding() .withCodec(CodecFactory.deflateCodec(9))); writePipeline.run(); PAssert.that( readPipeline.apply(AvroIO.read(GenericClass.class).from(outputFile.getAbsolutePath()))) .containsInAnyOrder(values); readPipeline.run(); try (DataFileStream dataFileStream = new DataFileStream(new FileInputStream(outputFile), new GenericDatumReader())) { assertEquals("deflate", dataFileStream.getMetaString("avro.codec")); } }
"Write first", AvroIO.write(GenericClass.class) .to(tmpFolder.getRoot().getAbsolutePath() + "/first") .withNumShards(2) .withWindowedWrites()); readPipeline .apply("Sequence second", GenerateSequence.from(3).to(7).withRate(1, Duration.millis(300))) "Write second", AvroIO.write(GenericClass.class) .to(tmpFolder.getRoot().getAbsolutePath() + "/second") .withNumShards(3) .withWindowedWrites());
.apply(AvroIO.write(GenericClass.class).to(outputFile.getAbsolutePath()).withoutSharding()); writePipeline.run();
"Write first", AvroIO.write(GenericClass.class) .to(tmpFolder.getRoot().getAbsolutePath() + "/first") .withNumShards(2)); writePipeline .apply("Create second", Create.of(secondValues)) "Write second", AvroIO.write(GenericClass.class) .to(tmpFolder.getRoot().getAbsolutePath() + "/second") .withNumShards(3)); writePipeline.run();
"Write File(s)", AvroIO.write(AvroPubsubMessageRecord.class) .to( new WindowedFilenamePolicy( options.getOutputDirectory(), options.getOutputShardTemplate(), options.getOutputFilenameSuffix())) .withTempDirectory(NestedValueProvider.of( options.getAvroTempDirectory(), (SerializableFunction<String, ResourceId>) input -> .withWindowedWrites() .withNumShards(options.getNumShards()));
public static PipelineResult run(Options options) { Pipeline pipeline = Pipeline.create(options); BigtableIO.Read read = BigtableIO.read() .withProjectId(options.getBigtableProjectId()) .withInstanceId(options.getBigtableInstanceId()) .withTableId(options.getBigtableTableId()); // Do not validate input fields if it is running as a template. if (options.as(DataflowPipelineOptions.class).getTemplateLocation() != null) { read = read.withoutValidation(); } ValueProvider<String> filePathPrefix = DualInputNestedValueProvider.of( options.getOutputDirectory(), options.getFilenamePrefix(), new SerializableFunction<TranslatorInput<String, String>, String>() { @Override public String apply(TranslatorInput<String, String> input) { return new StringBuilder(input.getX()).append(input.getY()).toString(); } }); pipeline .apply("Read from Bigtable", read) .apply("Transform to Avro", MapElements.via(new BigtableToAvroFn())) .apply( "Write to Avro in GCS", AvroIO.write(BigtableRow.class).to(filePathPrefix).withSuffix(".avro")); return pipeline.run(); }
@Test @SuppressWarnings("unchecked") @Category(NeedsRunner.class) public void testMetadata() throws Exception { List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); writePipeline .apply(Create.of(values)) .apply( AvroIO.write(GenericClass.class) .to(outputFile.getAbsolutePath()) .withoutSharding() .withMetadata( ImmutableMap.of( "stringKey", "stringValue", "longKey", 100L, "bytesKey", "bytesValue".getBytes(Charsets.UTF_8)))); writePipeline.run(); try (DataFileStream dataFileStream = new DataFileStream(new FileInputStream(outputFile), new GenericDatumReader())) { assertEquals("stringValue", dataFileStream.getMetaString("stringKey")); assertEquals(100L, dataFileStream.getMetaLong("longKey")); assertArrayEquals("bytesValue".getBytes(Charsets.UTF_8), dataFileStream.getMeta("bytesKey")); } }
/** * Tests that {@code AvroIO} can read an upgraded version of an old class, as long as the schema * resolution process succeeds. This test covers the case when a new, {@code @Nullable} field has * been added. * * <p>For more information, see http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution */ @Test @Category(NeedsRunner.class) public void testWriteThenReadSchemaUpgrade() throws Throwable { List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); writePipeline .apply(Create.of(values)) .apply(AvroIO.write(GenericClass.class).to(outputFile.getAbsolutePath()).withoutSharding()); writePipeline.run(); List<GenericClassV2> expected = ImmutableList.of(new GenericClassV2(3, "hi", null), new GenericClassV2(5, "bar", null)); PAssert.that( readPipeline.apply( AvroIO.read(GenericClassV2.class).from(outputFile.getAbsolutePath()))) .containsInAnyOrder(expected); readPipeline.run(); }
@Test public void testWriteDisplayData() { AvroIO.Write<GenericClass> write = AvroIO.write(GenericClass.class) .to("/foo") .withShardNameTemplate("-SS-of-NN-") .withSuffix("bar") .withNumShards(100) .withCodec(CodecFactory.deflateCodec(6)); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("filePrefix", "/foo")); assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-")); assertThat(displayData, hasDisplayItem("fileSuffix", "bar")); assertThat( displayData, hasDisplayItem( "schema", "{\"type\":\"record\",\"name\":\"GenericClass\",\"namespace\":\"org.apache.beam.sdk.io" + ".AvroIOTest$\",\"fields\":[{\"name\":\"intField\",\"type\":\"int\"}," + "{\"name\":\"stringField\",\"type\":\"string\"}]}")); assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("codec", CodecFactory.deflateCodec(6).toString())); } }
@Test @SuppressWarnings("unchecked") @Category(NeedsRunner.class) public void testWriteThenReadASingleFileWithNullCodec() throws Throwable { List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); writePipeline .apply(Create.of(values)) .apply( AvroIO.write(GenericClass.class) .to(outputFile.getAbsolutePath()) .withoutSharding() .withCodec(CodecFactory.nullCodec())); writePipeline.run(); PAssert.that( readPipeline.apply(AvroIO.read(GenericClass.class).from(outputFile.getAbsolutePath()))) .containsInAnyOrder(values); readPipeline.run(); try (DataFileStream dataFileStream = new DataFileStream(new FileInputStream(outputFile), new GenericDatumReader())) { assertEquals("null", dataFileStream.getMetaString("avro.codec")); } }
@Test @Category(NeedsRunner.class) public void testWriteThenReadJavaClass() throws Throwable { List<GenericClass> values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); writePipeline .apply(Create.of(values)) .apply( AvroIO.write(GenericClass.class) .to(writePipeline.newProvider(outputFile.getAbsolutePath())) .withoutSharding()); writePipeline.run(); PAssert.that( readPipeline.apply( "Read", AvroIO.read(GenericClass.class) .from(readPipeline.newProvider(outputFile.getAbsolutePath())))) .containsInAnyOrder(values); readPipeline.run(); }
private <T extends GenericRecord> void testWriteThenReadGeneratedClass( AvroIO.Write<T> writeTransform, AvroIO.Read<T> readTransform) throws Exception { File outputFile = tmpFolder.newFile("output.avro"); List<T> values = ImmutableList.of( (T) new AvroGeneratedUser("Bob", 256, null), (T) new AvroGeneratedUser("Alice", 128, null), (T) new AvroGeneratedUser("Ted", null, "white")); writePipeline .apply(Create.of(values)) .apply( writeTransform .to(writePipeline.newProvider(outputFile.getAbsolutePath())) .withoutSharding()); writePipeline.run(); PAssert.that( readPipeline.apply( "Read", readTransform.from(readPipeline.newProvider(outputFile.getAbsolutePath())))) .containsInAnyOrder(values); readPipeline.run(); }
/** See {@link TypedWrite#withMetadata} . */ public Write<T> withMetadata(Map<String, Object> metadata) { return new Write<>(inner.withMetadata(metadata)); }
/** See {@link TypedWrite#withWindowedWrites}. */ public Write<T> withWindowedWrites() { return new Write<>(inner.withWindowedWrites()); }
/** See {@link TypedWrite#withoutSharding}. */ public Write<T> withoutSharding() { return new Write<>(inner.withoutSharding()); }
/** See {@link TypedWrite#withCodec}. */ public Write<T> withCodec(CodecFactory codec) { return new Write<>(inner.withCodec(codec)); }
/** See {@link TypedWrite#withSuffix}. */ public Write<T> withSuffix(String filenameSuffix) { return new Write<>(inner.withSuffix(filenameSuffix)); }
/** See {@link TypedWrite#to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) public Write<T> toResource(ValueProvider<ResourceId> outputPrefix) { return new Write<>( inner.toResource(outputPrefix).withFormatFunction(SerializableFunctions.identity())); }