| <!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> |
| <!-- NewPage --> |
| <html lang="en"> |
| <head> |
| <!-- Generated by javadoc --> |
| <title>FileIO (Apache Beam 2.38.0-SNAPSHOT)</title> |
| <link rel="stylesheet" type="text/css" href="../../../../../stylesheet.css" title="Style"> |
| <script type="text/javascript" src="../../../../../script.js"></script> |
| </head> |
| <body> |
| <script type="text/javascript"><!-- |
| try { |
| if (location.href.indexOf('is-external=true') == -1) { |
| parent.document.title="FileIO (Apache Beam 2.38.0-SNAPSHOT)"; |
| } |
| } |
| catch(err) { |
| } |
| //--> |
| var methods = {"i0":9,"i1":9,"i2":9,"i3":9,"i4":9}; |
| var tabs = {65535:["t0","All Methods"],1:["t1","Static Methods"],8:["t4","Concrete Methods"]}; |
| var altColor = "altColor"; |
| var rowColor = "rowColor"; |
| var tableTab = "tableTab"; |
| var activeTableTab = "activeTableTab"; |
| </script> |
| <noscript> |
| <div>JavaScript is disabled on your browser.</div> |
| </noscript> |
| <!-- ========= START OF TOP NAVBAR ======= --> |
| <div class="topNav"><a name="navbar.top"> |
| <!-- --> |
| </a> |
| <div class="skipNav"><a href="#skip.navbar.top" title="Skip navigation links">Skip navigation links</a></div> |
| <a name="navbar.top.firstrow"> |
| <!-- --> |
| </a> |
| <ul class="navList" title="Navigation"> |
| <li><a href="../../../../../overview-summary.html">Overview</a></li> |
| <li><a href="package-summary.html">Package</a></li> |
| <li class="navBarCell1Rev">Class</li> |
| <li><a href="package-tree.html">Tree</a></li> |
| <li><a href="../../../../../deprecated-list.html">Deprecated</a></li> |
| <li><a href="../../../../../index-all.html">Index</a></li> |
| <li><a href="../../../../../help-doc.html">Help</a></li> |
| </ul> |
| </div> |
| <div class="subNav"> |
| <ul class="navList"> |
| <li><a href="../../../../../org/apache/beam/sdk/io/FileBasedSource.Mode.html" title="enum in org.apache.beam.sdk.io"><span class="typeNameLink">Prev Class</span></a></li> |
| <li><a href="../../../../../org/apache/beam/sdk/io/FileIO.Match.html" title="class in org.apache.beam.sdk.io"><span class="typeNameLink">Next Class</span></a></li> |
| </ul> |
| <ul class="navList"> |
| <li><a href="../../../../../index.html?org/apache/beam/sdk/io/FileIO.html" target="_top">Frames</a></li> |
| <li><a href="FileIO.html" target="_top">No Frames</a></li> |
| </ul> |
| <ul class="navList" id="allclasses_navbar_top"> |
| <li><a href="../../../../../allclasses-noframe.html">All Classes</a></li> |
| </ul> |
| <div> |
| <script type="text/javascript"><!-- |
| allClassesLink = document.getElementById("allclasses_navbar_top"); |
| if(window==top) { |
| allClassesLink.style.display = "block"; |
| } |
| else { |
| allClassesLink.style.display = "none"; |
| } |
| //--> |
| </script> |
| </div> |
| <div> |
| <ul class="subNavList"> |
| <li>Summary: </li> |
| <li><a href="#nested.class.summary">Nested</a> | </li> |
| <li>Field | </li> |
| <li><a href="#constructor.summary">Constr</a> | </li> |
| <li><a href="#method.summary">Method</a></li> |
| </ul> |
| <ul class="subNavList"> |
| <li>Detail: </li> |
| <li>Field | </li> |
| <li><a href="#constructor.detail">Constr</a> | </li> |
| <li><a href="#method.detail">Method</a></li> |
| </ul> |
| </div> |
| <a name="skip.navbar.top"> |
| <!-- --> |
| </a></div> |
| <!-- ========= END OF TOP NAVBAR ========= --> |
| <!-- ======== START OF CLASS DATA ======== --> |
| <div class="header"> |
| <div class="subTitle">org.apache.beam.sdk.io</div> |
| <h2 title="Class FileIO" class="title">Class FileIO</h2> |
| </div> |
| <div class="contentContainer"> |
| <ul class="inheritance"> |
| <li>java.lang.Object</li> |
| <li> |
| <ul class="inheritance"> |
| <li>org.apache.beam.sdk.io.FileIO</li> |
| </ul> |
| </li> |
| </ul> |
| <div class="description"> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <hr> |
| <br> |
| <pre>public class <span class="typeNameLabel">FileIO</span> |
| extends java.lang.Object</pre> |
| <div class="block">General-purpose transforms for working with files: listing files (matching), reading and writing. |
| |
| <h2>Matching filepatterns</h2> |
| |
| <p><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a> and <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#matchAll--"><code>matchAll()</code></a> match filepatterns (respectively either a single |
| filepattern or a <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> thereof) and return the files that match them as <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollections</code></a> of <code>MatchResult.Metadata</code>. Configuration options for them are in |
| <a href="../../../../../org/apache/beam/sdk/io/FileIO.MatchConfiguration.html" title="class in org.apache.beam.sdk.io"><code>FileIO.MatchConfiguration</code></a> and include features such as treatment of filepatterns that don't |
| match anything and continuous incremental matching of filepatterns (watching for new files). |
| |
| <h3>Example: Watching a single filepattern for new files</h3> |
| |
| <p>This example matches a single filepattern repeatedly every 30 seconds, continuously returns |
| new matched files as an unbounded <code>PCollection<Metadata></code> and stops if no new files appear |
| for 1 hour. |
| |
| <pre><code> |
| PCollection<Metadata> matches = p.apply(FileIO.match() |
| .filepattern("...") |
| .continuously( |
| Duration.standardSeconds(30), afterTimeSinceNewOutput(Duration.standardHours(1)))); |
| </code></pre> |
| |
| <h3>Example: Matching a PCollection of filepatterns arriving from Kafka</h3> |
| |
| <p>This example reads filepatterns from Kafka and matches each one as it arrives, producing again |
| an unbounded <code>PCollection<Metadata></code>, and failing in case the filepattern doesn't match |
| anything. |
| |
| <pre><code> |
| PCollection<String> filepatterns = p.apply(KafkaIO.read()...); |
| |
| PCollection<Metadata> matches = filepatterns.apply(FileIO.matchAll() |
| .withEmptyMatchTreatment(DISALLOW)); |
| </code></pre> |
| |
| <h2>Reading files</h2> |
| |
| <p><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#readMatches--"><code>readMatches()</code></a> converts each result of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a> or <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#matchAll--"><code>matchAll()</code></a> to a <a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadableFile.html" title="class in org.apache.beam.sdk.io"><code>FileIO.ReadableFile</code></a> that is convenient for reading a file's contents, optionally decompressing it. |
| |
| <h3>Example: Returning filenames and contents of compressed files matching a filepattern</h3> |
| |
| <p>This example matches a single filepattern and returns <code>KVs</code> of filenames and their |
| contents as <code>String</code>, decompressing each file with GZIP. |
| |
| <pre><code> |
| PCollection<KV<String, String>> filesAndContents = p |
| .apply(FileIO.match().filepattern("hdfs://path/to/*.gz")) |
| // withCompression can be omitted - by default compression is detected from the filename. |
| .apply(FileIO.readMatches().withCompression(GZIP)) |
| .apply(MapElements |
| // uses imports from TypeDescriptors |
| .into(kvs(strings(), strings())) |
| .via((ReadableFile f) -> { |
| try { |
| return KV.of( |
| f.getMetadata().resourceId().toString(), f.readFullyAsUTF8String()); |
| } catch (IOException ex) { |
| throw new RuntimeException("Failed to read the file", ex); |
| } |
| })); |
| </code></pre> |
| |
| <h2>Writing files</h2> |
| |
| <p><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#write--"><code>write()</code></a> and <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#writeDynamic--"><code>writeDynamic()</code></a> write elements from a <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> of a given |
| type to files, using a given <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a> to write a set of elements to each file. The collection |
| can be bounded or unbounded - in either case, writing happens by default per window and pane, and |
| the amount of data in each window and pane is finite, so a finite number of files ("shards") are |
| written for each window and pane. There are several aspects to this process: |
| |
| <ul> |
| <li><b>How many shards are generated per pane:</b> This is controlled by <i>sharding</i>, using |
| <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withNumShards-int-"><code>FileIO.Write.withNumShards(int)</code></a> or <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withSharding-org.apache.beam.sdk.transforms.PTransform-"><code>FileIO.Write.withSharding(org.apache.beam.sdk.transforms.PTransform<org.apache.beam.sdk.values.PCollection<UserT>, org.apache.beam.sdk.values.PCollectionView<java.lang.Integer>>)</code></a>. The default is runner-specific, |
| so the number of shards will vary based on runner behavior, though at least 1 shard will |
| always be produced for every non-empty pane. Note that setting a fixed number of shards can |
| hurt performance: it adds an additional <a href="../../../../../org/apache/beam/sdk/transforms/GroupByKey.html" title="class in org.apache.beam.sdk.transforms"><code>GroupByKey</code></a> to the pipeline. However, it is |
| required to set it when writing an unbounded <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> due to <a |
| href="https://issues.apache.org/jira/browse/BEAM-1438">BEAM-1438</a> and similar behavior |
| in other runners. |
| <li><b>How the shards are named:</b> This is controlled by a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.FileNaming.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Write.FileNaming</code></a>: |
| filenames can depend on a variety of inputs, e.g. the window, the pane, total number of |
| shards, the current file's shard index, and compression. Controlling the file naming is |
| described in the section <i>File naming</i> below. |
| <li><b>Which elements go into which shard:</b> Elements within a pane get distributed into |
| different shards created for that pane arbitrarily, though <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io"><code>FileIO.Write</code></a> attempts to |
| make shards approximately evenly sized. For more control over which elements go into which |
| files, consider using <i>dynamic destinations</i> (see below). |
| <li><b>How a given set of elements is written to a shard:</b> This is controlled by the <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a>, e.g. <a href="../../../../../org/apache/beam/sdk/io/AvroIO.html#sink-java.lang.Class-"><code>AvroIO.sink(java.lang.Class<ElementT>)</code></a> will generate Avro files. The <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a> controls the |
| format of a single file: how to open a file, how to write each element to it, and how to |
| close the file - but it does not control the set of files or which elements go where. |
| Elements are written to a shard in an arbitrary order. <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io"><code>FileIO.Write</code></a> can |
| additionally compress the generated files using <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withCompression-org.apache.beam.sdk.io.Compression-"><code>FileIO.Write.withCompression(org.apache.beam.sdk.io.Compression)</code></a>. |
| <li><b>How all of the above can be element-dependent:</b> This is controlled by <i>dynamic |
| destinations</i>. It is possible to have different groups of elements use different |
| policies for naming files and for configuring the <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a>. See "dynamic destinations" |
| below. |
| </ul> |
| |
| <h3>File naming</h3> |
| |
| <p>The names of generated files are produced by a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.FileNaming.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Write.FileNaming</code></a>. The default naming |
| strategy is to name files in the format: <code>$prefix-$start-$end-$pane-$shard-of-$numShards$suffix$compressionSuffix</code>, where: |
| |
| <ul> |
| <li>$prefix is set by <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withPrefix-java.lang.String-"><code>FileIO.Write.withPrefix(java.lang.String)</code></a>, the default is "output". |
| <li>$start and $end are boundaries of the window of data being written, formatted in ISO 8601 |
| format (YYYY-mm-ddTHH:MM:SSZZZ). The window is omitted in case this is the global window. |
| <li>$pane is the index of the pane within the window. The pane is omitted in case it is known |
| to be the only pane for this window. |
| <li>$shard is the index of the current shard being written, out of the $numShards total shards |
| written for the current pane. Both are formatted using 5 digits (or more if necessary |
| according to $numShards) and zero-padded. |
| <li>$suffix is set by <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withSuffix-java.lang.String-"><code>FileIO.Write.withSuffix(java.lang.String)</code></a>, the default is empty. |
| <li>$compressionSuffix is based on the default extension for the chosen <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withCompression-org.apache.beam.sdk.io.Compression-"><code>compression type</code></a>. |
| </ul> |
| |
| <p>For example: <code>data-2017-12-01T19:00:00Z-2017-12-01T20:00:00Z-2-00010-of-00050.txt.gz</code> |
| |
| <p>Alternatively, one can specify a custom naming strategy using <code>Write#withNaming(Write.FileNaming)</code>. |
| |
| <p>If <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#to-java.lang.String-"><code>FileIO.Write.to(java.lang.String)</code></a> is specified, then the filenames produced by the <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.FileNaming.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Write.FileNaming</code></a> |
| are resolved relative to that directory. |
| |
| <p>When using dynamic destinations via <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#writeDynamic--"><code>writeDynamic()</code></a> (see below), specifying a custom |
| naming strategy is required, using <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withNaming-org.apache.beam.sdk.transforms.SerializableFunction-"><code>FileIO.Write.withNaming(SerializableFunction)</code></a> or <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withNaming-org.apache.beam.sdk.transforms.Contextful-"><code>FileIO.Write.withNaming(Contextful)</code></a>. In those, pass a function that creates a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.FileNaming.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Write.FileNaming</code></a> |
| for the requested group ("destination"). You can either implement a custom <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.FileNaming.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Write.FileNaming</code></a>, or use <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#defaultNaming-java.lang.String-java.lang.String-"><code>FileIO.Write.defaultNaming(java.lang.String, java.lang.String)</code></a> to configure the default naming strategy |
| with a prefix and suffix as per above. |
| |
| <h3>Dynamic destinations</h3> |
| |
| <p>If the elements in the input collection can be partitioned into groups that should be treated |
| differently, <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io"><code>FileIO.Write</code></a> supports different treatment per group ("destination"). It can |
| use different file naming strategies for different groups, and can differently configure the |
| <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a>, e.g. write different elements to Avro files in different directories with different |
| schemas. |
| |
| <p>This feature is supported by <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#writeDynamic--"><code>writeDynamic()</code></a>. Use <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#by-org.apache.beam.sdk.transforms.SerializableFunction-"><code>FileIO.Write.by(org.apache.beam.sdk.transforms.SerializableFunction<UserT, DestinationT>)</code></a> to specify how to |
| partition the elements into groups ("destinations"). Then elements will be grouped by |
| destination, and <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withNaming-org.apache.beam.sdk.transforms.Contextful-"><code>FileIO.Write.withNaming(Contextful)</code></a> and <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#via-org.apache.beam.sdk.transforms.Contextful-"><code>FileIO.Write.via(Contextful)</code></a> will be |
| applied separately within each group, i.e. different groups will be written using the file naming |
| strategies returned by <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withNaming-org.apache.beam.sdk.transforms.Contextful-"><code>FileIO.Write.withNaming(Contextful)</code></a> and using sinks returned by <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#via-org.apache.beam.sdk.transforms.Contextful-"><code>FileIO.Write.via(Contextful)</code></a> for the respective destinations. Note that currently sharding can not be |
| destination-dependent: every window/pane for every destination will use the same number of shards |
| specified via <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withNumShards-int-"><code>FileIO.Write.withNumShards(int)</code></a> or <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#withSharding-org.apache.beam.sdk.transforms.PTransform-"><code>FileIO.Write.withSharding(org.apache.beam.sdk.transforms.PTransform<org.apache.beam.sdk.values.PCollection<UserT>, org.apache.beam.sdk.values.PCollectionView<java.lang.Integer>>)</code></a>. |
| |
| <h3>Writing custom types to sinks</h3> |
| |
| <p>Normally, when writing a collection of a custom type using a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a> that takes a |
| different type (for example, writing a <code>PCollection<Event></code> to a text-based <code>Sink<String></code>), one can simply apply a <code>ParDo</code> or <code>MapElements</code> to convert the custom |
| type to the sink's <i>output type</i>. |
| |
| <p>However, when using dynamic destinations, in many such cases the destination needs to be |
| extracted from the original type, so such a conversion is not possible. For example, one might |
| write events of a custom class <code>Event</code> to a text sink, using the event's "type" as a |
| destination. In that case, specify an <i>output function</i> in <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html#via-org.apache.beam.sdk.transforms.Contextful-org.apache.beam.sdk.transforms.Contextful-"><code>FileIO.Write.via(Contextful, |
| Contextful)</code></a> or <code>Write#via(Contextful, Sink)</code>. |
| |
| <h3>Example: Writing CSV files</h3> |
| |
| <pre><code> |
| class CSVSink implements FileIO.Sink<List<String>> { |
| private String header; |
| private PrintWriter writer; |
| |
| public CSVSink(List<String> colNames) { |
| this.header = Joiner.on(",").join(colNames); |
| } |
| |
| public void open(WritableByteChannel channel) throws IOException { |
| writer = new PrintWriter(Channels.newOutputStream(channel)); |
| writer.println(header); |
| } |
| |
| public void write(List<String> element) throws IOException { |
| writer.println(Joiner.on(",").join(element)); |
| } |
| |
| public void flush() throws IOException { |
| writer.flush(); |
| } |
| } |
| |
| PCollection<BankTransaction> transactions = ...; |
| // Convert transactions to strings before writing them to the CSV sink. |
| transactions.apply(MapElements |
| .into(TypeDescriptors.lists(TypeDescriptors.strings())) |
| .via(tx -> Arrays.asList(tx.getUser(), tx.getAmount()))) |
| .apply(FileIO.<List<String>>write() |
| .via(new CSVSink(Arrays.asList("user", "amount"))) |
| .to(".../path/to/") |
| .withPrefix("transactions") |
| .withSuffix(".csv")); |
| </code></pre> |
| |
| <h3>Example: Writing CSV files to different directories and with different headers</h3> |
| |
| <pre><code> |
| enum TransactionType { |
| DEPOSIT, |
| WITHDRAWAL, |
| TRANSFER, |
| ... |
| |
| List<String> getFieldNames(); |
| List<String> getAllFields(BankTransaction tx); |
| } |
| |
| PCollection<BankTransaction> transactions = ...; |
| transactions.apply(FileIO.<TransactionType, Transaction>writeDynamic() |
| .by(Transaction::getTypeName) |
| .via(tx -> tx.getTypeName().toFields(tx), // Convert the data to be written to CSVSink |
| type -> new CSVSink(type.getFieldNames())) |
| .to(".../path/to/") |
| .withNaming(type -> defaultNaming(type + "-transactions", ".csv")); |
| </code></pre></div> |
| </li> |
| </ul> |
| </div> |
| <div class="summary"> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <!-- ======== NESTED CLASS SUMMARY ======== --> |
| <ul class="blockList"> |
| <li class="blockList"><a name="nested.class.summary"> |
| <!-- --> |
| </a> |
| <h3>Nested Class Summary</h3> |
| <table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Nested Class Summary table, listing nested classes, and an explanation"> |
| <caption><span>Nested Classes</span><span class="tabEnd"> </span></caption> |
| <tr> |
| <th class="colFirst" scope="col">Modifier and Type</th> |
| <th class="colLast" scope="col">Class and Description</th> |
| </tr> |
| <tr class="altColor"> |
| <td class="colFirst"><code>static class </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.Match.html" title="class in org.apache.beam.sdk.io">FileIO.Match</a></span></code> |
| <div class="block">Implementation of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a>.</div> |
| </td> |
| </tr> |
| <tr class="rowColor"> |
| <td class="colFirst"><code>static class </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.MatchAll.html" title="class in org.apache.beam.sdk.io">FileIO.MatchAll</a></span></code> |
| <div class="block">Implementation of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#matchAll--"><code>matchAll()</code></a>.</div> |
| </td> |
| </tr> |
| <tr class="altColor"> |
| <td class="colFirst"><code>static class </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.MatchConfiguration.html" title="class in org.apache.beam.sdk.io">FileIO.MatchConfiguration</a></span></code> |
| <div class="block">Describes configuration for matching filepatterns, such as <a href="../../../../../org/apache/beam/sdk/io/fs/EmptyMatchTreatment.html" title="enum in org.apache.beam.sdk.io.fs"><code>EmptyMatchTreatment</code></a> and |
| continuous watching for matching files.</div> |
| </td> |
| </tr> |
| <tr class="rowColor"> |
| <td class="colFirst"><code>static class </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadableFile.html" title="class in org.apache.beam.sdk.io">FileIO.ReadableFile</a></span></code> |
| <div class="block">A utility class for accessing a potentially compressed file.</div> |
| </td> |
| </tr> |
| <tr class="altColor"> |
| <td class="colFirst"><code>static class </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadMatches.html" title="class in org.apache.beam.sdk.io">FileIO.ReadMatches</a></span></code> |
| <div class="block">Implementation of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#readMatches--"><code>readMatches()</code></a>.</div> |
| </td> |
| </tr> |
| <tr class="rowColor"> |
| <td class="colFirst"><code>static interface </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io">FileIO.Sink</a><<a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="type parameter in FileIO.Sink">ElementT</a>></span></code> |
| <div class="block">Specifies how to write elements to individual files in <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#write--"><code>write()</code></a> and <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#writeDynamic--"><code>writeDynamic()</code></a>.</div> |
| </td> |
| </tr> |
| <tr class="altColor"> |
| <td class="colFirst"><code>static class </code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io">FileIO.Write</a><<a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="type parameter in FileIO.Write">DestinationT</a>,<a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="type parameter in FileIO.Write">UserT</a>></span></code> |
| <div class="block">Implementation of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#write--"><code>write()</code></a> and <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#writeDynamic--"><code>writeDynamic()</code></a>.</div> |
| </td> |
| </tr> |
| </table> |
| </li> |
| </ul> |
| <!-- ======== CONSTRUCTOR SUMMARY ======== --> |
| <ul class="blockList"> |
| <li class="blockList"><a name="constructor.summary"> |
| <!-- --> |
| </a> |
| <h3>Constructor Summary</h3> |
| <table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Constructor Summary table, listing constructors, and an explanation"> |
| <caption><span>Constructors</span><span class="tabEnd"> </span></caption> |
| <tr> |
| <th class="colOne" scope="col">Constructor and Description</th> |
| </tr> |
| <tr class="altColor"> |
| <td class="colOne"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#FileIO--">FileIO</a></span>()</code> </td> |
| </tr> |
| </table> |
| </li> |
| </ul> |
| <!-- ========== METHOD SUMMARY =========== --> |
| <ul class="blockList"> |
| <li class="blockList"><a name="method.summary"> |
| <!-- --> |
| </a> |
| <h3>Method Summary</h3> |
| <table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Method Summary table, listing methods, and an explanation"> |
| <caption><span id="t0" class="activeTableTab"><span>All Methods</span><span class="tabEnd"> </span></span><span id="t1" class="tableTab"><span><a href="javascript:show(1);">Static Methods</a></span><span class="tabEnd"> </span></span><span id="t4" class="tableTab"><span><a href="javascript:show(8);">Concrete Methods</a></span><span class="tabEnd"> </span></span></caption> |
| <tr> |
| <th class="colFirst" scope="col">Modifier and Type</th> |
| <th class="colLast" scope="col">Method and Description</th> |
| </tr> |
| <tr id="i0" class="altColor"> |
| <td class="colFirst"><code>static <a href="../../../../../org/apache/beam/sdk/io/FileIO.Match.html" title="class in org.apache.beam.sdk.io">FileIO.Match</a></code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--">match</a></span>()</code> |
| <div class="block">Matches a filepattern using <a href="../../../../../org/apache/beam/sdk/io/FileSystems.html#match-java.util.List-"><code>FileSystems.match(java.util.List<java.lang.String>)</code></a> and produces a collection of matched |
| resources (both files and directories) as <code>MatchResult.Metadata</code>.</div> |
| </td> |
| </tr> |
| <tr id="i1" class="rowColor"> |
| <td class="colFirst"><code>static <a href="../../../../../org/apache/beam/sdk/io/FileIO.MatchAll.html" title="class in org.apache.beam.sdk.io">FileIO.MatchAll</a></code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#matchAll--">matchAll</a></span>()</code> |
| <div class="block">Like <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a>, but matches each filepattern in a collection of filepatterns.</div> |
| </td> |
| </tr> |
| <tr id="i2" class="altColor"> |
| <td class="colFirst"><code>static <a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadMatches.html" title="class in org.apache.beam.sdk.io">FileIO.ReadMatches</a></code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#readMatches--">readMatches</a></span>()</code> |
| <div class="block">Converts each result of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a> or <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#matchAll--"><code>matchAll()</code></a> to a <a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadableFile.html" title="class in org.apache.beam.sdk.io"><code>FileIO.ReadableFile</code></a> which can |
| be used to read the contents of each file, optionally decompressing it.</div> |
| </td> |
| </tr> |
| <tr id="i3" class="rowColor"> |
| <td class="colFirst"><code>static <InputT> <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io">FileIO.Write</a><java.lang.Void,InputT></code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#write--">write</a></span>()</code> |
| <div class="block">Writes elements to files using a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a>.</div> |
| </td> |
| </tr> |
| <tr id="i4" class="altColor"> |
| <td class="colFirst"><code>static <DestT,InputT><br><a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io">FileIO.Write</a><DestT,InputT></code></td> |
| <td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/io/FileIO.html#writeDynamic--">writeDynamic</a></span>()</code> |
| <div class="block">Writes elements to files using a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a> and grouping the elements using "dynamic |
| destinations".</div> |
| </td> |
| </tr> |
| </table> |
| <ul class="blockList"> |
| <li class="blockList"><a name="methods.inherited.from.class.java.lang.Object"> |
| <!-- --> |
| </a> |
| <h3>Methods inherited from class java.lang.Object</h3> |
| <code>clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait</code></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </div> |
| <div class="details"> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <!-- ========= CONSTRUCTOR DETAIL ======== --> |
| <ul class="blockList"> |
| <li class="blockList"><a name="constructor.detail"> |
| <!-- --> |
| </a> |
| <h3>Constructor Detail</h3> |
| <a name="FileIO--"> |
| <!-- --> |
| </a> |
| <ul class="blockListLast"> |
| <li class="blockList"> |
| <h4>FileIO</h4> |
| <pre>public FileIO()</pre> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| <!-- ============ METHOD DETAIL ========== --> |
| <ul class="blockList"> |
| <li class="blockList"><a name="method.detail"> |
| <!-- --> |
| </a> |
| <h3>Method Detail</h3> |
| <a name="match--"> |
| <!-- --> |
| </a> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <h4>match</h4> |
| <pre>public static <a href="../../../../../org/apache/beam/sdk/io/FileIO.Match.html" title="class in org.apache.beam.sdk.io">FileIO.Match</a> match()</pre> |
| <div class="block">Matches a filepattern using <a href="../../../../../org/apache/beam/sdk/io/FileSystems.html#match-java.util.List-"><code>FileSystems.match(java.util.List<java.lang.String>)</code></a> and produces a collection of matched |
| resources (both files and directories) as <code>MatchResult.Metadata</code>. |
| |
| <p>By default, matches the filepattern once and produces a bounded <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>. To |
| continuously watch the filepattern for new matches, use <code>MatchAll#continuously(Duration, |
| TerminationCondition)</code> - this will produce an unbounded <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>. |
| |
| <p>By default, a filepattern matching no resources is treated according to <a href="../../../../../org/apache/beam/sdk/io/fs/EmptyMatchTreatment.html#DISALLOW"><code>EmptyMatchTreatment.DISALLOW</code></a>. To configure this behavior, use <a href="../../../../../org/apache/beam/sdk/io/FileIO.Match.html#withEmptyMatchTreatment-org.apache.beam.sdk.io.fs.EmptyMatchTreatment-"><code>FileIO.Match.withEmptyMatchTreatment(org.apache.beam.sdk.io.fs.EmptyMatchTreatment)</code></a>. |
| |
| <p>Returned <code>MatchResult.Metadata</code> are deduplicated by filename. For example, if this |
| transform observes a file with the same name several times with different metadata (e.g. |
| because the file is growing), it will emit the metadata the first time this file is observed, |
| and will ignore future changes to this file.</div> |
| </li> |
| </ul> |
| <a name="matchAll--"> |
| <!-- --> |
| </a> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <h4>matchAll</h4> |
| <pre>public static <a href="../../../../../org/apache/beam/sdk/io/FileIO.MatchAll.html" title="class in org.apache.beam.sdk.io">FileIO.MatchAll</a> matchAll()</pre> |
| <div class="block">Like <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a>, but matches each filepattern in a collection of filepatterns. |
| |
| <p>Resources are not deduplicated between filepatterns, i.e. if the same resource matches |
| multiple filepatterns, it will be produced multiple times. |
| |
| <p>By default, a filepattern matching no resources is treated according to <a href="../../../../../org/apache/beam/sdk/io/fs/EmptyMatchTreatment.html#ALLOW_IF_WILDCARD"><code>EmptyMatchTreatment.ALLOW_IF_WILDCARD</code></a>. To configure this behavior, use <a href="../../../../../org/apache/beam/sdk/io/FileIO.MatchAll.html#withEmptyMatchTreatment-org.apache.beam.sdk.io.fs.EmptyMatchTreatment-"><code>FileIO.MatchAll.withEmptyMatchTreatment(org.apache.beam.sdk.io.fs.EmptyMatchTreatment)</code></a>.</div> |
| </li> |
| </ul> |
| <a name="readMatches--"> |
| <!-- --> |
| </a> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <h4>readMatches</h4> |
| <pre>public static <a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadMatches.html" title="class in org.apache.beam.sdk.io">FileIO.ReadMatches</a> readMatches()</pre> |
| <div class="block">Converts each result of <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#match--"><code>match()</code></a> or <a href="../../../../../org/apache/beam/sdk/io/FileIO.html#matchAll--"><code>matchAll()</code></a> to a <a href="../../../../../org/apache/beam/sdk/io/FileIO.ReadableFile.html" title="class in org.apache.beam.sdk.io"><code>FileIO.ReadableFile</code></a> which can |
| be used to read the contents of each file, optionally decompressing it.</div> |
| </li> |
| </ul> |
| <a name="write--"> |
| <!-- --> |
| </a> |
| <ul class="blockList"> |
| <li class="blockList"> |
| <h4>write</h4> |
| <pre>public static <InputT> <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io">FileIO.Write</a><java.lang.Void,InputT> write()</pre> |
| <div class="block">Writes elements to files using a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a>. See class-level documentation.</div> |
| </li> |
| </ul> |
| <a name="writeDynamic--"> |
| <!-- --> |
| </a> |
| <ul class="blockListLast"> |
| <li class="blockList"> |
| <h4>writeDynamic</h4> |
| <pre>public static <DestT,InputT> <a href="../../../../../org/apache/beam/sdk/io/FileIO.Write.html" title="class in org.apache.beam.sdk.io">FileIO.Write</a><DestT,InputT> writeDynamic()</pre> |
| <div class="block">Writes elements to files using a <a href="../../../../../org/apache/beam/sdk/io/FileIO.Sink.html" title="interface in org.apache.beam.sdk.io"><code>FileIO.Sink</code></a> and grouping the elements using "dynamic |
| destinations". See class-level documentation.</div> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </div> |
| </div> |
| <!-- ========= END OF CLASS DATA ========= --> |
| <!-- ======= START OF BOTTOM NAVBAR ====== --> |
| <div class="bottomNav"><a name="navbar.bottom"> |
| <!-- --> |
| </a> |
| <div class="skipNav"><a href="#skip.navbar.bottom" title="Skip navigation links">Skip navigation links</a></div> |
| <a name="navbar.bottom.firstrow"> |
| <!-- --> |
| </a> |
| <ul class="navList" title="Navigation"> |
| <li><a href="../../../../../overview-summary.html">Overview</a></li> |
| <li><a href="package-summary.html">Package</a></li> |
| <li class="navBarCell1Rev">Class</li> |
| <li><a href="package-tree.html">Tree</a></li> |
| <li><a href="../../../../../deprecated-list.html">Deprecated</a></li> |
| <li><a href="../../../../../index-all.html">Index</a></li> |
| <li><a href="../../../../../help-doc.html">Help</a></li> |
| </ul> |
| </div> |
| <div class="subNav"> |
| <ul class="navList"> |
| <li><a href="../../../../../org/apache/beam/sdk/io/FileBasedSource.Mode.html" title="enum in org.apache.beam.sdk.io"><span class="typeNameLink">Prev Class</span></a></li> |
| <li><a href="../../../../../org/apache/beam/sdk/io/FileIO.Match.html" title="class in org.apache.beam.sdk.io"><span class="typeNameLink">Next Class</span></a></li> |
| </ul> |
| <ul class="navList"> |
| <li><a href="../../../../../index.html?org/apache/beam/sdk/io/FileIO.html" target="_top">Frames</a></li> |
| <li><a href="FileIO.html" target="_top">No Frames</a></li> |
| </ul> |
| <ul class="navList" id="allclasses_navbar_bottom"> |
| <li><a href="../../../../../allclasses-noframe.html">All Classes</a></li> |
| </ul> |
| <div> |
| <script type="text/javascript"><!-- |
| allClassesLink = document.getElementById("allclasses_navbar_bottom"); |
| if(window==top) { |
| allClassesLink.style.display = "block"; |
| } |
| else { |
| allClassesLink.style.display = "none"; |
| } |
| //--> |
| </script> |
| </div> |
| <div> |
| <ul class="subNavList"> |
| <li>Summary: </li> |
| <li><a href="#nested.class.summary">Nested</a> | </li> |
| <li>Field | </li> |
| <li><a href="#constructor.summary">Constr</a> | </li> |
| <li><a href="#method.summary">Method</a></li> |
| </ul> |
| <ul class="subNavList"> |
| <li>Detail: </li> |
| <li>Field | </li> |
| <li><a href="#constructor.detail">Constr</a> | </li> |
| <li><a href="#method.detail">Method</a></li> |
| </ul> |
| </div> |
| <a name="skip.navbar.bottom"> |
| <!-- --> |
| </a></div> |
| <!-- ======== END OF BOTTOM NAVBAR ======= --> |
| </body> |
| </html> |