Class CsvIO
- java.lang.Object
-
- org.apache.beam.sdk.io.csv.CsvIO
-
public class CsvIO extends java.lang.ObjectPTransforms for reading and writing CSV files.Reading CSV files
Reading from CSV files is not yet implemented. Please see https://github.com/apache/beam/issues/24552.
Valid CSVFormat Configuration
A
CSVFormatmust meet the following conditions to be considered valid when reading CSV:String[]header - must contain at least one column name, and all column names must be non-empty.booleanallowDuplicateHeaderNames - must be false.booleanallowMissingColumnNames - must be false.booleanignoreHeaderCase - must be false.booleanskipHeaderRecord - must be false. The header is already accounted for during parsing.
Ignored CSVFormat parameters
The following
CSVFormatparameters are either not relevant for parsing CSV or are validated satisfactorily by the Apache Commons CSV library.booleanautoFlushcharcommentMarkerchardelimitercharescapecharquoteorg.apache.commons.csv.QuoteModequoteModeStringnullStringcharrecordSeparator- systemRecordSeparator
- firstRecordAsHeader
java.lang.Object...headerCommentsbooleanignoreEmptyLinesbooleanignoreSurroundingSpacesbooleantrimbooleanskipHeaderRecordbooleantrailingDelimiter
Writing CSV files
To write a
PCollectionto one or more CSV files, useCsvIO.Write, usingwriteRows(java.lang.String, org.apache.commons.csv.CSVFormat)orwrite(java.lang.String, org.apache.commons.csv.CSVFormat).CsvIO.Writesupports writingRowor custom Java types using an inferredSchema. Examples below show both scenarios. See the Beam Programming Guide on inferring schemas for more information on how to enable Beam to infer aSchemafrom a custom Java type.CsvIO.Writeonly supports writing the parts ofSchemaaware types that do not contain any nestedSchema.FieldTypes such aSchema.TypeName.ROWor repeatedSchema.TypeName.ARRAYtypes. SeeVALID_FIELD_TYPE_SETfor validSchema.FieldTypes.Example usage:
Suppose we have the following
Transactionclass annotated with@DefaultSchema(JavaBeanSchema.class)so that Beam can infer itsSchema:@DefaultSchema(JavaBeanSchema.class) public class Transaction { public Transaction() { … } public Long getTransactionId(); public void setTransactionId(Long transactionId) { … } public String getBank() { … } public void setBank(String bank) { … } public double getPurchaseAmount() { … } public void setPurchaseAmount(double purchaseAmount) { … } }From a
PCollection<Transaction>,CsvIO.Writecan write one or many CSV files automatically creating the header based on its inferredSchema.PCollection<Transaction> transactions = ... transactions.apply(CsvIO.<Transaction>write("path/to/folder/prefix", CSVFormat.DEFAULT));The resulting CSV files will look like the following where the header is repeated for every file, whereas by default,
CsvIO.Writewill write all fields in sorted order of the field names.bank,purchaseAmount,transactionId A,10.23,12345 B,54.65,54321 C,11.76,98765To control the order and subset of fields that
CsvIO.Writewrites, useCSVFormat.withHeader(java.lang.Class<? extends java.lang.Enum<?>>). Note, however, the following constraints:- Each header column must match a field name in the
Schema; matching is case sensitive. - Matching header columns must match
Schemafields that are validSchema.FieldTypes; seeVALID_FIELD_TYPE_SET. CSVFormatonly allows repeated header columns whenCSVFormat.withAllowDuplicateHeaderNames()
The following example shows the use of
CSVFormat.withHeader(java.lang.Class<? extends java.lang.Enum<?>>)to control the order and subset ofTransactionfields.PCollection<Transaction> transactions ... transactions.apply( CsvIO .<Transaction>write("path/to/folder/prefix", CSVFormat.DEFAULT.withHeader("transactionId", "purchaseAmount")) );The resulting CSV files will look like the following where the header is repeated for every file, but will only include the subset of fields in their listed order.
transactionId,purchaseAmount 12345,10.23 54321,54.65 98765,11.76In addition to header customization,
CsvIO.WritesupportsCSVFormat.withHeaderComments(java.lang.Object...)as shown below. Note thatCSVFormat.withCommentMarker(char)is required when specifying header comments.PCollection<Transaction> transactions = ... transactions .apply( CsvIO.<Transaction>write("path/to/folder/prefix", CSVFormat.DEFAULT .withCommentMarker('#') .withHeaderComments("Bank Report", "1970-01-01", "Operator: John Doe") );The resulting CSV files will look like the following where the header and header comments are repeated for every shard file.
# Bank Report # 1970-01-01 # Operator: John Doe bank,purchaseAmount,transactionId A,10.23,12345 B,54.65,54321 C,11.76,98765A
PCollectionofRows works just like custom Java types illustrated above, except we usewriteRows(java.lang.String, org.apache.commons.csv.CSVFormat)as shown below for the sameTransactionclass. We deriveTransaction'sSchemausing aDefaultSchema.DefaultSchemaProvider. Note that hard-coding theRows below is for illustration purposes. Developers are instead encouraged to take advantage ofDefaultSchema.DefaultSchemaProvider.toRowFunction(org.apache.beam.sdk.values.TypeDescriptor<T>).DefaultSchemaProvider defaultSchemaProvider = new DefaultSchemaProvider(); Schema schema = defaultSchemaProvider.schemaFor(TypeDescriptor.of(Transaction.class)); PCollection<Row> transactions = pipeline.apply(Create.of( Row .withSchema(schema) .withFieldValue("bank", "A") .withFieldValue("purchaseAmount", 10.23) .withFieldValue("transactionId", "12345") .build(), Row .withSchema(schema) .withFieldValue("bank", "B") .withFieldValue("purchaseAmount", 54.65) .withFieldValue("transactionId", "54321") .build(), Row .withSchema(schema) .withFieldValue("bank", "C") .withFieldValue("purchaseAmount", 11.76) .withFieldValue("transactionId", "98765") .build() ); transactions.apply( CsvIO .writeRowsTo("gs://bucket/path/to/folder/prefix", CSVFormat.DEFAULT) );Writing the transactions
PCollectionofRows would yield the following CSV file content.bank,purchaseAmount,transactionId A,10.23,12345 B,54.65,54321 C,11.76,98765CsvIO.Writedoes not support the followingCSVFormatproperties and will throw anIllegalArgumentException.CSVFormat.withAllowMissingColumnNames()CSVFormat.withAutoFlush(boolean)CSVFormat.withIgnoreHeaderCase()CSVFormat.withIgnoreSurroundingSpaces()
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static classCsvIO.Write<T>PTransformfor writing CSV files.
-
Field Summary
Fields Modifier and Type Field Description static java.util.Set<org.apache.beam.sdk.schemas.Schema.FieldType>VALID_FIELD_TYPE_SETThe validSchema.FieldTypefrom whichCsvIOconverts CSV records to the fields.
-
Constructor Summary
Constructors Constructor Description CsvIO()
-
Method Summary
All Methods Static Methods Concrete Methods Modifier and Type Method Description static <T> CsvIOParse<T>parse(java.lang.Class<T> klass, org.apache.commons.csv.CSVFormat csvFormat)Instantiates aCsvIOParsefor parsing CSV string records into customSchema-mappedClass<T>es from the records' assumed CsvFormat.static CsvIOParse<org.apache.beam.sdk.values.Row>parseRows(org.apache.beam.sdk.schemas.Schema schema, org.apache.commons.csv.CSVFormat csvFormat)Instantiates aCsvIOParsefor parsing CSV string records intoRows from the records' assumed CsvFormat and expectedSchema.static <T> CsvIO.Write<T>write(java.lang.String to, org.apache.commons.csv.CSVFormat csvFormat)Instantiates aCsvIO.Writefor writing user types inCSVFormatformat.static CsvIO.Write<org.apache.beam.sdk.values.Row>writeRows(java.lang.String to, org.apache.commons.csv.CSVFormat csvFormat)
-
-
-
Field Detail
-
VALID_FIELD_TYPE_SET
public static final java.util.Set<org.apache.beam.sdk.schemas.Schema.FieldType> VALID_FIELD_TYPE_SET
The validSchema.FieldTypefrom whichCsvIOconverts CSV records to the fields.Schema.FieldType.BYTESchema.FieldType.BOOLEANSchema.FieldType.DATETIMESchema.FieldType.DECIMALSchema.FieldType.DOUBLESchema.FieldType.INT16Schema.FieldType.INT32Schema.FieldType.INT64Schema.FieldType.FLOATSchema.FieldType.STRING
-
-
Method Detail
-
write
public static <T> CsvIO.Write<T> write(java.lang.String to, org.apache.commons.csv.CSVFormat csvFormat)
Instantiates aCsvIO.Writefor writing user types inCSVFormatformat.
-
writeRows
public static CsvIO.Write<org.apache.beam.sdk.values.Row> writeRows(java.lang.String to, org.apache.commons.csv.CSVFormat csvFormat)
-
parse
public static <T> CsvIOParse<T> parse(java.lang.Class<T> klass, org.apache.commons.csv.CSVFormat csvFormat)
Instantiates aCsvIOParsefor parsing CSV string records into customSchema-mappedClass<T>es from the records' assumed CsvFormat. See the Beam Programming Guide on how to configure your customClass<T>for Beam to infer itsSchemausing aSchemaProviderannotation such asAutoValueSchemaorJavaBeanSchema.Example usage
The example below illustrates parsing CsvFormat#DEFAULT formatted CSV string records, read fromTextIO.Read, into anAutoValueSchemaannotated AutoValue data classPCollection.{@code // SomeDataClass is a data class configured for Beam to automatically infer its Schema.
-
parseRows
public static CsvIOParse<org.apache.beam.sdk.values.Row> parseRows(org.apache.beam.sdk.schemas.Schema schema, org.apache.commons.csv.CSVFormat csvFormat)
Instantiates aCsvIOParsefor parsing CSV string records intoRows from the records' assumed CsvFormat and expectedSchema.Example usage
The example below illustrates parsing CsvFormat#DEFAULT formatted CSV string records, read fromTextIO.Read, into aRowPCollection.// Define the expected Schema. Schema schema = Schema.of( Schema.Field.of("someString", FieldType.STRING), Schema.Field.of("someInteger", FieldType.INT32) ); // Pipeline example reads CSV string records from Google Cloud storage and writes to BigQuery. Pipeline pipeline = Pipeline.create(); // Read CSV records from Google Cloud storage using TextIO. PCollection<String> csvRecords = pipeline .apply(TextIO.read().from("gs://bucket/folder/*.csv"); // Apply the CSV records PCollection<String> to the CsvIOParse transform instantiated using CsvIO.parseRows. CsvIOParseResult<Row> result = csvRecords.apply(CsvIO.parseRow( schema, CsvFormat.DEFAULT.withHeader("someString", "someInteger") )); // Acquire any processing errors to either write to logs or apply to a downstream dead letter queue such as BigQuery. result.getErrors().apply(BigQueryIO.<CsvIOParseError>write() .to("project:dataset.table_of_errors") .useBeamSchema() .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(WriteDisposition.WRITE_APPEND)); // Acquire the successful PCollection<Row> output. PCollection<Row> output = result.getOutput(); // Do something with the output such as write to BigQuery. output.apply(BigQueryIO.<Row>write() .to("project:dataset.table_of_output") .useBeamSchema() .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(WriteDisposition.WRITE_APPEND));
-
-