.to(new TestDynamicDestinations(baseDir, schemaView)) .withTempDirectory(baseDir); break; case WITHOUT_SHARDING: write = write.withoutSharding(); break; case FIXED_3_SHARDS: write = write.withNumShards(3); break; default:
@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); }
.apply( AvroIO.<Long, GenericClass>writeCustomType() .to(writePipeline.newProvider(outputFile.getAbsolutePath())) .withFormatFunction(new CreateGenericClass()) .withSchema(ReflectData.get().getSchema(GenericClass.class)) .withoutSharding()); writePipeline.run();
"Store Avro files", AvroIO.<Struct>writeCustomTypeToGenericRecords() .to(new SchemaBasedDynamicDestinations(avroSchemas, outputDirectoryName, resource)) .withTempDirectory(resource));
@Override public WriteFilesResult<DestinationT> expand(PCollection<UserT> input) { checkArgument( getFilenamePrefix() != null || getTempDirectory() != null, "Need to set either the filename prefix or the tempDirectory of a AvroIO.Write " + "transform."); if (getFilenamePolicy() != null) { checkArgument( getShardTemplate() == null && getFilenameSuffix() == null, "shardTemplate and filenameSuffix should only be used with the default " + "filename policy"); if (getDynamicDestinations() != null) { checkArgument( getFormatFunction() == null, "A format function should not be specified " + "with DynamicDestinations. Use DynamicDestinations.formatRecord instead"); } else { checkArgument( getSchema() != null, "Unless using DynamicDestinations, .withSchema() is required."); ValueProvider<ResourceId> tempDirectory = getTempDirectory(); if (tempDirectory == null) { tempDirectory = getFilenamePrefix(); new AvroSink<>(tempDirectory, resolveDynamicDestinations(), getGenericRecords())); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); if (getWindowedWrites()) {
DynamicAvroDestinations<UserT, DestinationT, OutputT> resolveDynamicDestinations() { DynamicAvroDestinations<UserT, DestinationT, OutputT> dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { // In this case DestinationT is Void. FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); if (usedFilenamePolicy == null) { usedFilenamePolicy = DefaultFilenamePolicy.fromStandardParameters( getFilenamePrefix(), getShardTemplate(), getFilenameSuffix(), getWindowedWrites()); } dynamicDestinations = (DynamicAvroDestinations<UserT, DestinationT, OutputT>) constantDestinations( usedFilenamePolicy, getSchema(), getMetadata(), getCodec().getCodec(), getFormatFunction()); } return dynamicDestinations; }
/** See {@link TypedWrite#withoutSharding}. */ public Write<T> withoutSharding() { return new Write<>(inner.withoutSharding()); }
/** * Writes to file(s) with the given output prefix. See {@link FileSystems} for information on * supported file systems. * * <p>The name of the output files will be determined by the {@link FilenamePolicy} used. * * <p>By default, a {@link DefaultFilenamePolicy} will build output filenames using the * specified prefix, a shard name template (see {@link #withShardNameTemplate(String)}, and a * common suffix (if supplied using {@link #withSuffix(String)}). This default can be overridden * using {@link #to(FilenamePolicy)}. */ public TypedWrite<UserT, DestinationT, OutputT> to(String outputPrefix) { return to(FileBasedSink.convertToFileResourceIfPossible(outputPrefix)); }
@Test @SuppressWarnings("unchecked") public void testWriteWithSerDeCustomXZCodec() throws Exception { AvroIO.Write<String> write = AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.xzCodec(9)); assertEquals( CodecFactory.xzCodec(9).toString(), SerializableUtils.clone(write.inner.getCodec()).getCodec().toString()); }
@Test public void testWriteWithCustomCodec() throws Exception { AvroIO.Write<String> write = AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.snappyCodec()); assertEquals(SNAPPY_CODEC, write.inner.getCodec().toString()); }
@Test public void testWriteWithDefaultCodec() throws Exception { AvroIO.Write<String> write = AvroIO.write(String.class).to("/tmp/foo/baz"); assertEquals(CodecFactory.snappyCodec().toString(), write.inner.getCodec().toString()); }
/** See {@link TypedWrite#withTempDirectory(ValueProvider)}. */ @Experimental(Kind.FILESYSTEM) public Write<T> withTempDirectory(ValueProvider<ResourceId> tempDirectory) { return new Write<>(inner.withTempDirectory(tempDirectory)); }
/** Set the base directory used to generate temporary files. */ @Experimental(Kind.FILESYSTEM) public TypedWrite<UserT, DestinationT, OutputT> withTempDirectory(ResourceId tempDirectory) { return withTempDirectory(StaticValueProvider.of(tempDirectory)); }
@Test @SuppressWarnings("unchecked") public void testWriteWithSerDeCustomDeflateCodec() throws Exception { AvroIO.Write<String> write = AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.deflateCodec(9)); assertEquals( CodecFactory.deflateCodec(9).toString(), SerializableUtils.clone(write.inner.getCodec()).getCodec().toString()); }
/** See {@link TypedWrite#to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) public Write<T> toResource(ValueProvider<ResourceId> outputPrefix) { return new Write<>( inner.toResource(outputPrefix).withFormatFunction(SerializableFunctions.identity())); }
/** See {@link TypedWrite#withTempDirectory(ResourceId)}. */ public Write<T> withTempDirectory(ResourceId tempDirectory) { return new Write<>(inner.withTempDirectory(tempDirectory)); }
/** * Forces a single file as output and empty shard name template. This option is only compatible * with unwindowed writes. * * <p>For unwindowed writes, constraining the number of shards is likely to reduce the * performance of a pipeline. Setting this value is not recommended unless you require a * specific number of output files. * * <p>This is equivalent to {@code .withNumShards(1).withShardNameTemplate("")} */ public TypedWrite<UserT, DestinationT, OutputT> withoutSharding() { return withNumShards(1).withShardNameTemplate(""); }
/** * See {@link TypedWrite#to(DynamicAvroDestinations)}. * * @deprecated Use {@link FileIO#write()} or {@link FileIO#writeDynamic()} instead. */ @Deprecated public Write<T> to(DynamicAvroDestinations<T, ?, T> dynamicDestinations) { return new Write<>(inner.to(dynamicDestinations).withFormatFunction(null)); }
/** See {@link TypedWrite#to(FilenamePolicy)}. */ public Write<T> to(FilenamePolicy filenamePolicy) { return new Write<>( inner.to(filenamePolicy).withFormatFunction(SerializableFunctions.identity())); }
/** See {@link TypedWrite#to(ResourceId)} . */ @Experimental(Kind.FILESYSTEM) public Write<T> to(ResourceId outputPrefix) { return new Write<>( inner.to(outputPrefix).withFormatFunction(SerializableFunctions.identity())); }