@DefaultAnnotation(value=org.checkerframework.checker.nullness.qual.NonNull.class)
See: Description
| Interface | Description | 
|---|---|
| AvroIO.RecordFormatter<ElementT> | Deprecated Users can achieve the same by providing this transform in a  ParDobefore using write in AvroIOAvroIO.write(Class). | 
| AvroSink.DatumWriterFactory<T> | |
| AvroSource.DatumReaderFactory<T> | |
| CompressedSource.DecompressingChannelFactory | Factory interface for creating channels that decompress the content of an underlying channel. | 
| FileBasedSink.OutputFileHints | Provides hints about how to generate output files, such as a suggested filename suffix (e.g. | 
| FileBasedSink.WritableByteChannelFactory | Implementations create instances of  WritableByteChannelused byFileBasedSinkand related classes to allow decorating, or otherwise transforming, the raw data that
 would normally be written directly to theWritableByteChannelpassed intoFileBasedSink.WritableByteChannelFactory.create(WritableByteChannel). | 
| FileIO.Sink<ElementT> | Specifies how to write elements to individual files in  FileIO.write()andFileIO.writeDynamic(). | 
| FileIO.Write.FileNaming | A policy for generating names for shard files. | 
| FileSystemRegistrar | A registrar that creates  FileSysteminstances fromPipelineOptions. | 
| ShardingFunction<UserT,DestinationT> | Function for assigning  ShardedKeys to input elements for shardedWriteFiles. | 
| TextRowCountEstimator.SamplingStrategy | Sampling Strategy shows us when should we stop reading further files. | 
| UnboundedSource.CheckpointMark | A marker representing the progress and state of an  UnboundedSource.UnboundedReader. | 
| Class | Description | 
|---|---|
| AvroIO | Deprecated Avro related classes are deprecated in module  beam-sdks-java-coreand
     will be eventually removed. | 
| AvroIO.Parse<T> | |
| AvroIO.ParseAll<T> | Deprecated See  AvroIO.parseAllGenericRecords(SerializableFunction)for details. | 
| AvroIO.ParseFiles<T> | |
| AvroIO.Read<T> | Implementation of  AvroIO.read(java.lang.Class<T>)andAvroIO.readGenericRecords(org.apache.avro.Schema). | 
| AvroIO.ReadAll<T> | Deprecated See  AvroIO.readAll(Class)for details. | 
| AvroIO.ReadFiles<T> | Implementation of  AvroIO.readFiles(java.lang.Class<T>). | 
| AvroIO.Sink<ElementT> | |
| AvroIO.TypedWrite<UserT,DestinationT,OutputT> | Implementation of  AvroIO.write(java.lang.Class<T>). | 
| AvroIO.Write<T> | This class is used as the default return value of  AvroIO.write(java.lang.Class<T>) | 
| AvroSchemaIOProvider | Deprecated Avro related classes are deprecated in module  beam-sdks-java-coreand
     will be eventually removed. | 
| AvroSink<UserT,DestinationT,OutputT> | Deprecated Avro related classes are deprecated in module  beam-sdks-java-coreand
     will be eventually removed. | 
| AvroSource<T> | Deprecated Avro related classes are deprecated in module  beam-sdks-java-coreand
     will be eventually removed. | 
| AvroSource.AvroReader<T> | A  BlockBasedSource.BlockBasedReaderfor reading blocks from Avro files. | 
| BlockBasedSource<T> | A  BlockBasedSourceis aFileBasedSourcewhere a file consists of blocks of
 records. | 
| BlockBasedSource.Block<T> | A  Blockrepresents a block of records that can be read. | 
| BlockBasedSource.BlockBasedReader<T> | A  Readerthat reads records from aBlockBasedSource. | 
| BoundedReadFromUnboundedSource<T> | PTransformthat reads a bounded amount of data from anUnboundedSource, specified
 as one or both of a maximum number of elements or a maximum period of time to read. | 
| BoundedSource<T> | A  Sourcethat reads a finite amount of input and, because of that, supports some
 additional operations. | 
| BoundedSource.BoundedReader<T> | A  Readerthat reads a bounded amount of input and supports some additional operations,
 such as progress estimation and dynamic work rebalancing. | 
| ClassLoaderFileSystem | A read-only  FileSystemimplementation looking up resources using a ClassLoader. | 
| ClassLoaderFileSystem.ClassLoaderFileSystemRegistrar | AutoServiceregistrar for theClassLoaderFileSystem. | 
| ClassLoaderFileSystem.ClassLoaderResourceId | |
| CompressedSource<T> | A Source that reads from compressed files. | 
| CompressedSource.CompressedReader<T> | Reader for a  CompressedSource. | 
| CountingSource | Most users should use  GenerateSequenceinstead. | 
| CountingSource.CounterMark | The checkpoint for an unbounded  CountingSourceis simply the last value produced. | 
| DefaultFilenamePolicy | A default  FileBasedSink.FilenamePolicyfor windowed and unwindowed files. | 
| DefaultFilenamePolicy.Params | Encapsulates constructor parameters to  DefaultFilenamePolicy. | 
| DefaultFilenamePolicy.ParamsCoder | A Coder for  DefaultFilenamePolicy.Params. | 
| DynamicAvroDestinations<UserT,DestinationT,OutputT> | Deprecated Avro related classes are deprecated in module  beam-sdks-java-coreand
     will be eventually removed. | 
| DynamicFileDestinations | Some helper classes that derive from  FileBasedSink.DynamicDestinations. | 
| FileBasedSink<UserT,DestinationT,OutputT> | Abstract class for file-based output. | 
| FileBasedSink.DynamicDestinations<UserT,DestinationT,OutputT> | A class that allows value-dependent writes in  FileBasedSink. | 
| FileBasedSink.FilenamePolicy | A naming policy for output files. | 
| FileBasedSink.FileResult<DestinationT> | Result of a single bundle write. | 
| FileBasedSink.FileResultCoder<DestinationT> | A coder for  FileBasedSink.FileResultobjects. | 
| FileBasedSink.WriteOperation<DestinationT,OutputT> | Abstract operation that manages the process of writing to  FileBasedSink. | 
| FileBasedSink.Writer<DestinationT,OutputT> | Abstract writer that writes a bundle to a  FileBasedSink. | 
| FileBasedSource<T> | A common base class for all file-based  Sources. | 
| FileBasedSource.FileBasedReader<T> | A  readerthat implements code common to readers ofFileBasedSources. | 
| FileIO | General-purpose transforms for working with files: listing files (matching), reading and writing. | 
| FileIO.Match | Implementation of  FileIO.match(). | 
| FileIO.MatchAll | Implementation of  FileIO.matchAll(). | 
| FileIO.MatchConfiguration | Describes configuration for matching filepatterns, such as  EmptyMatchTreatmentand
 continuous watching for matching files. | 
| FileIO.ReadableFile | A utility class for accessing a potentially compressed file. | 
| FileIO.ReadMatches | Implementation of  FileIO.readMatches(). | 
| FileIO.Write<DestinationT,UserT> | Implementation of  FileIO.write()andFileIO.writeDynamic(). | 
| FileSystem<ResourceIdT extends ResourceId> | File system interface in Beam. | 
| FileSystems | Clients facing  FileSystemutility. | 
| FileSystemUtils | |
| GenerateSequence | A  PTransformthat produces longs starting from the given value, and either up to the
 given limit or untilLong.MAX_VALUE/ until the given time elapses. | 
| GenerateSequence.External | Exposes GenerateSequence as an external transform for cross-language usage. | 
| GenerateSequence.External.ExternalConfiguration | Parameters class to expose the transform to an external SDK. | 
| LocalFileSystemRegistrar | AutoServiceregistrar for theLocalFileSystem. | 
| LocalResources | Helper functions for producing a  ResourceIdthat references a local file or directory. | 
| OffsetBasedSource<T> | A  BoundedSourcethat uses offsets to define starting and ending positions. | 
| OffsetBasedSource.OffsetBasedReader<T> | A  Source.Readerthat implements code common to readers of allOffsetBasedSources. | 
| Read | A  PTransformfor reading from aSource. | 
| Read.Bounded<T> | PTransformthat reads from aBoundedSource. | 
| Read.Builder | Helper class for building  Readtransforms. | 
| Read.Unbounded<T> | PTransformthat reads from aUnboundedSource. | 
| ReadableFileCoder | A  CoderforFileIO.ReadableFile. | 
| ReadAllViaFileBasedSource<T> | Reads each file in the input  PCollectionofFileIO.ReadableFileusing given parameters
 for splitting files into offset ranges and for creating aFileBasedSourcefor a file. | 
| ReadAllViaFileBasedSource.ReadFileRangesFnExceptionHandler | A class to handle errors which occur during file reads. | 
| ReadAllViaFileBasedSourceTransform<InT,T> | |
| ReadAllViaFileBasedSourceTransform.AbstractReadFileRangesFn<InT,T> | |
| ReadAllViaFileBasedSourceTransform.SplitIntoRangesFn | |
| ReadAllViaFileBasedSourceWithFilename<T> | Reads each file of the input  PCollectionand outputs each element as the value of aKV, where the key is the filename from which that value came. | 
| ShardNameTemplate | Standard shard naming templates. | 
| Source<T> | Base class for defining input formats and creating a  Sourcefor reading the input. | 
| Source.Reader<T> | The interface that readers of custom input sources must implement. | 
| TextIO | PTransforms for reading and writing text files. | 
| TextIO.Read | Implementation of  TextIO.read(). | 
| TextIO.ReadAll | Deprecated See  TextIO.readAll()for details. | 
| TextIO.ReadFiles | Implementation of  TextIO.readFiles(). | 
| TextIO.Sink | Implementation of  TextIO.sink(). | 
| TextIO.TypedWrite<UserT,DestinationT> | Implementation of  TextIO.write(). | 
| TextIO.Write | This class is used as the default return value of  TextIO.write(). | 
| TextRowCountEstimator | This returns a row count estimation for files associated with a file pattern. | 
| TextRowCountEstimator.Builder | Builder for  TextRowCountEstimator. | 
| TextRowCountEstimator.LimitNumberOfFiles | This strategy stops sampling if we sample enough number of bytes. | 
| TextRowCountEstimator.LimitNumberOfTotalBytes | This strategy stops sampling when total number of sampled bytes are more than some threshold. | 
| TextRowCountEstimator.SampleAllFiles | This strategy samples all the files. | 
| TextSource | Implementation detail of  TextIO.Read. | 
| TFRecordIO | PTransforms for reading and writing TensorFlow TFRecord files. | 
| TFRecordIO.Read | Implementation of  TFRecordIO.read(). | 
| TFRecordIO.ReadFiles | Implementation of  TFRecordIO.readFiles(). | 
| TFRecordIO.Sink | |
| TFRecordIO.Write | Implementation of  TFRecordIO.write(). | 
| UnboundedSource<OutputT,CheckpointMarkT extends UnboundedSource.CheckpointMark> | A  Sourcethat reads an unbounded amount of input and, because of that, supports some
 additional operations such as checkpointing, watermarks, and record ids. | 
| UnboundedSource.CheckpointMark.NoopCheckpointMark | A checkpoint mark that does nothing when finalized. | 
| UnboundedSource.UnboundedReader<OutputT> | A  Readerthat reads an unbounded amount of input. | 
| WriteFiles<UserT,DestinationT,OutputT> | A  PTransformthat writes to aFileBasedSink. | 
| WriteFilesResult<DestinationT> | The result of a  WriteFilestransform. | 
| Enum | Description | 
|---|---|
| CompressedSource.CompressionMode | Deprecated Use  Compressioninstead | 
| Compression | Various compression types for reading/writing files. | 
| FileBasedSink.CompressionType | Deprecated use  Compression. | 
| FileBasedSource.Mode | A given  FileBasedSourcerepresents a file resource of one of these types. | 
| FileIO.ReadMatches.DirectoryTreatment | Enum to control how directories are handled. | 
| TextIO.CompressionType | Deprecated Use  Compression. | 
| TFRecordIO.CompressionType | Deprecated Use  Compression. | 
| Exception | Description | 
|---|---|
| TextRowCountEstimator.NoEstimationException | An exception that will be thrown if the estimator cannot get an estimation of the number of
 lines. | 
AvroIO, and TextIO.
 The classes in this package provide Read transforms that create PCollections from
 existing storage:
 
 PCollection<TableRow> inputData = pipeline.apply(
     BigQueryIO.readTableRows().from("clouddataflow-readonly:samples.weather_stations"));
 Write transforms that persist PCollections to external storage:
 
 PCollection<Integer> numbers = ...;
 numbers.apply(TextIO.write().to("gs://my_bucket/path/to/numbers"));