private void runTestWrite(String[] elems, String... base64) throws IOException { File tmpFile = Files.createTempFile(tempFolder.getRoot().toPath(), "file", ".tfrecords").toFile(); String filename = tmpFile.getPath(); PCollection<byte[]> input = writePipeline .apply(Create.of(Arrays.asList(elems))) .apply(ParDo.of(new StringToByteArray())); TFRecordIO.Write write = TFRecordIO.write().to(filename).withoutSharding(); input.apply(write); writePipeline.run(); FileInputStream fis = new FileInputStream(tmpFile); String written = BaseEncoding.base64().encode(ByteStreams.toByteArray(fis)); // bytes written may vary depending the order of elems assertThat(written, isIn(base64)); }
/** @deprecated use {@link #withCompression}. */ @Deprecated public Write withCompressionType(CompressionType compressionType) { return withCompression(compressionType.canonical); }
"Write via TFRecordIO.write", TFRecordIO.write() .to(baseFilenameViaWrite) .withNumShards(numShards) .withSuffix(suffix) .withCompression(writeCompression));
@Test public void testWriteDisplayData() { TFRecordIO.Write write = TFRecordIO.write() .to("/foo") .withSuffix("bar") .withShardNameTemplate("-SS-of-NN-") .withNumShards(100) .withCompression(GZIP); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("filePrefix", "/foo")); assertThat(displayData, hasDisplayItem("fileSuffix", "bar")); assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-")); assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("compressionType", GZIP.toString())); }
@Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .add(DisplayData.item("filePrefix", getOutputPrefix()).withLabel("Output File Prefix")) .addIfNotNull( DisplayData.item("fileSuffix", getFilenameSuffix()).withLabel("Output File Suffix")) .addIfNotNull( DisplayData.item("shardNameTemplate", getShardTemplate()) .withLabel("Output Shard Name Template")) .addIfNotDefault( DisplayData.item("numShards", getNumShards()).withLabel("Maximum Output Shards"), 0) .add( DisplayData.item("compressionType", getCompression().toString()) .withLabel("Compression Type")); } }
@Override public PDone expand(PCollection<byte[]> input) { checkState( getOutputPrefix() != null, "need to set the output prefix of a TFRecordIO.Write transform"); WriteFiles<byte[], Void, byte[]> write = WriteFiles.to( new TFRecordSink( getOutputPrefix(), getShardTemplate(), getFilenameSuffix(), getCompression())); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); } input.apply("Write", write); return PDone.in(input.getPipeline()); }
/** * Writes to the provided number of shards. * * <p>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. * * @param numShards the number of shards to use, or 0 to let the system decide. * @see ShardNameTemplate */ public Write withNumShards(int numShards) { checkArgument(numShards >= 0, "Number of shards %s must be >= 0", numShards); return toBuilder().setNumShards(numShards).build(); }
/** * Writes TFRecord file(s) with the given output prefix. The {@code prefix} will be used as a to * generate a {@link ResourceId} using any supported {@link FileSystem}. * * <p>In addition to their prefix, created files will have a shard identifier (see {@link * #withNumShards(int)}), and end in a common suffix, if given by {@link #withSuffix(String)}. * * <p>For more information on filenames, see {@link DefaultFilenamePolicy}. */ public Write to(String outputPrefix) { return to(FileBasedSink.convertToFileResourceIfPossible(outputPrefix)); }
/** * Writes to output files using the specified compression type. * * <p>If no compression type is specified, the default is {@link Compression#UNCOMPRESSED}. See * {@link TFRecordIO.Read#withCompression} for more details. */ public Write withCompression(Compression compression) { return toBuilder().setCompression(compression).build(); }
/** * Writes to the file(s) with the given filename suffix. * * @see ShardNameTemplate */ public Write withSuffix(String suffix) { return toBuilder().setFilenameSuffix(suffix).build(); }
/** * Forces a single file as output. * * <p>Constraining the number of shards is likely to reduce the performance of a pipeline. Using * this setting is not recommended unless you truly require a single output file. * * <p>This is a shortcut for {@code .withNumShards(1).withShardNameTemplate("")} */ public Write withoutSharding() { return withNumShards(1).withShardNameTemplate(""); }
/** * Uses the given shard name template. * * @see ShardNameTemplate */ public Write withShardNameTemplate(String shardTemplate) { return toBuilder().setShardTemplate(shardTemplate).build(); }
/** Like {@link #to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) public Write toResource(ValueProvider<ResourceId> outputResource) { return toBuilder().setOutputPrefix(outputResource).build(); }