@VisibleForTesting TypedRead<T> withTestServices(BigQueryServices testServices) { return toBuilder().setBigQueryServices(testServices).build(); } }
/** * Reads from a BigQuery table or query and returns a {@link PCollection} with one element per * each row of the table or query result, parsed from the BigQuery AVRO format using the specified * function. * * <p>Each {@link SchemaAndRecord} contains a BigQuery {@link TableSchema} and a {@link * GenericRecord} representing the row, indexed by column name. Here is a sample parse function * that parses click events from a table. * * <pre>{@code * class ClickEvent { long userId; String url; ... } * * p.apply(BigQueryIO.read(new SerializableFunction<SchemaAndRecord, ClickEvent>() { * public ClickEvent apply(SchemaAndRecord record) { * GenericRecord r = record.getRecord(); * return new ClickEvent((Long) r.get("userId"), (String) r.get("url")); * } * }).from("..."); * }</pre> */ public static <T> TypedRead<T> read(SerializableFunction<SchemaAndRecord, T> parseFn) { return new AutoValue_BigQueryIO_TypedRead.Builder<T>() .setValidate(true) .setWithTemplateCompatibility(false) .setBigQueryServices(new BigQueryServicesImpl()) .setParseFn(parseFn) .build(); }
/** See {@link QueryPriority}. */ public TypedRead<T> withQueryPriority(QueryPriority priority) { return toBuilder().setQueryPriority(priority).build(); }
/** See {@link Read#withoutResultFlattening()}. */ public TypedRead<T> withoutResultFlattening() { return toBuilder().setFlattenResults(false).build(); }
@Experimental(Experimental.Kind.SOURCE_SINK) public TypedRead<T> withTemplateCompatibility() { return toBuilder().setWithTemplateCompatibility(true).build(); }
/** See {@link Read#fromQuery(ValueProvider)}. */ public TypedRead<T> fromQuery(ValueProvider<String> query) { ensureFromNotCalledYet(); return toBuilder().setQuery(query).setFlattenResults(true).setUseLegacySql(true).build(); }
/** * Sets a {@link Coder} for the result of the parse function. This may be required if a coder * can not be inferred automatically. */ public TypedRead<T> withCoder(Coder<T> coder) { return toBuilder().setCoder(coder).build(); }
/** * BigQuery geographic location where the query <a * href="https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs">job</a> will be * executed. If not specified, Beam tries to determine the location by examining the tables * referenced by the query. Location must be specified for queries not executed in US or EU. See * <a href="https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs/query">BigQuery Jobs: * query</a>. */ public TypedRead<T> withQueryLocation(String location) { return toBuilder().setQueryLocation(location).build(); }
/** See {@link Read#from(ValueProvider)}. */ public TypedRead<T> from(ValueProvider<String> tableSpec) { ensureFromNotCalledYet(); return toBuilder() .setJsonTableRef( NestedValueProvider.of( NestedValueProvider.of(tableSpec, new TableSpecToTableRef()), new TableRefToJson())) .build(); }
/** See {@link Read#withoutValidation()}. */ public TypedRead<T> withoutValidation() { return toBuilder().setValidate(false).build(); }
/** See {@link Read#usingStandardSql()}. */ public TypedRead<T> usingStandardSql() { return toBuilder().setUseLegacySql(false).build(); }