/** * Writes to Avro file(s) with the specified metadata. * * <p>Supported value types are String, Long, and byte[]. */ public TypedWrite<UserT, DestinationT, OutputT> withMetadata(Map<String, Object> metadata) { Map<String, String> badKeys = Maps.newLinkedHashMap(); for (Map.Entry<String, Object> entry : metadata.entrySet()) { Object v = entry.getValue(); if (!(v instanceof String || v instanceof Long || v instanceof byte[])) { badKeys.put(entry.getKey(), v.getClass().getSimpleName()); } } checkArgument( badKeys.isEmpty(), "Metadata value type must be one of String, Long, or byte[]. Found {}", badKeys); return toBuilder().setMetadata(ImmutableMap.copyOf(metadata)).build(); }
/** * Specifies a format function to convert {@link UserT} to the output type. If {@link * #to(DynamicAvroDestinations)} is used, {@link DynamicAvroDestinations#formatRecord} must be * used instead. */ public TypedWrite<UserT, DestinationT, OutputT> withFormatFunction( @Nullable SerializableFunction<UserT, OutputT> formatFunction) { return toBuilder().setFormatFunction(formatFunction).build(); }
/** * Use a {@link DynamicAvroDestinations} object to vend {@link FilenamePolicy} objects. These * objects can examine the input record when creating a {@link FilenamePolicy}. A directory for * temporary files must be specified using {@link #withTempDirectory}. * * @deprecated Use {@link FileIO#write()} or {@link FileIO#writeDynamic()} instead. */ @Experimental(Kind.FILESYSTEM) @Deprecated public <NewDestinationT> TypedWrite<UserT, NewDestinationT, OutputT> to( DynamicAvroDestinations<UserT, NewDestinationT, OutputT> dynamicDestinations) { return toBuilder() .setDynamicDestinations((DynamicAvroDestinations) dynamicDestinations) .build(); }
/** * Configures the number of output shards produced overall (when using unwindowed writes) or * per-window (when using windowed 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. * * @param numShards the number of shards to use, or 0 to let the system decide. */ public TypedWrite<UserT, DestinationT, OutputT> withNumShards(int numShards) { checkArgument(numShards >= 0); return toBuilder().setNumShards(numShards).build(); }
/** Writes to Avro file(s) compressed using specified codec. */ public TypedWrite<UserT, DestinationT, OutputT> withCodec(CodecFactory codec) { return toBuilder().setCodec(new SerializableAvroCodecFactory(codec)).build(); }
/** * Writes to files named according to the given {@link FileBasedSink.FilenamePolicy}. A * directory for temporary files must be specified using {@link #withTempDirectory}. */ @Experimental(Kind.FILESYSTEM) public TypedWrite<UserT, DestinationT, OutputT> to(FilenamePolicy filenamePolicy) { return toBuilder().setFilenamePolicy(filenamePolicy).build(); }
/** Like {@link #to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) public TypedWrite<UserT, DestinationT, OutputT> toResource( ValueProvider<ResourceId> outputPrefix) { return toBuilder().setFilenamePrefix(outputPrefix).build(); }
/** * Uses the given {@link ShardNameTemplate} for naming output files. This option may only be * used when using one of the default filename-prefix to() overrides. * * <p>See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are * used. */ public TypedWrite<UserT, DestinationT, OutputT> withShardNameTemplate(String shardTemplate) { return toBuilder().setShardTemplate(shardTemplate).build(); }
/** * Configures the filename suffix for written files. This option may only be used when using one * of the default filename-prefix to() overrides. * * <p>See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are * used. */ public TypedWrite<UserT, DestinationT, OutputT> withSuffix(String filenameSuffix) { return toBuilder().setFilenameSuffix(filenameSuffix).build(); }
/** * Sets the the output schema. Can only be used when the output type is {@link GenericRecord} * and when not using {@link #to(DynamicAvroDestinations)}. */ public TypedWrite<UserT, DestinationT, OutputT> withSchema(Schema schema) { return toBuilder().setSchema(schema).build(); }
/** * Preserves windowing of input elements and writes them to files based on the element's window. * * <p>If using {@link #to(FileBasedSink.FilenamePolicy)}. Filenames will be generated using * {@link FilenamePolicy#windowedFilename}. See also {@link WriteFiles#withWindowedWrites()}. */ public TypedWrite<UserT, DestinationT, OutputT> withWindowedWrites() { return toBuilder().setWindowedWrites(true).build(); }
/** Set the base directory used to generate temporary files. */ @Experimental(Kind.FILESYSTEM) public TypedWrite<UserT, DestinationT, OutputT> withTempDirectory( ValueProvider<ResourceId> tempDirectory) { return toBuilder().setTempDirectory(tempDirectory).build(); }