|
52 | 52 | import com.google.protobuf.DynamicMessage; |
53 | 53 | import com.google.protobuf.Message; |
54 | 54 | import java.io.IOException; |
| 55 | +import java.io.InputStream; |
| 56 | +import java.io.OutputStream; |
55 | 57 | import java.io.Serializable; |
56 | 58 | import java.lang.reflect.InvocationTargetException; |
| 59 | +import java.util.ArrayList; |
57 | 60 | import java.util.Arrays; |
58 | 61 | import java.util.Collections; |
59 | 62 | import java.util.List; |
|
74 | 77 | import org.apache.beam.sdk.PipelineRunner; |
75 | 78 | import org.apache.beam.sdk.coders.CannotProvideCoderException; |
76 | 79 | import org.apache.beam.sdk.coders.Coder; |
| 80 | +import org.apache.beam.sdk.coders.CoderException; |
77 | 81 | import org.apache.beam.sdk.coders.CoderRegistry; |
78 | 82 | import org.apache.beam.sdk.coders.KvCoder; |
79 | 83 | import org.apache.beam.sdk.coders.StringUtf8Coder; |
| 84 | +import org.apache.beam.sdk.coders.StructuredCoder; |
80 | 85 | import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; |
81 | 86 | import org.apache.beam.sdk.extensions.avro.io.AvroSource; |
82 | 87 | import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; |
|
118 | 123 | import org.apache.beam.sdk.transforms.MapElements; |
119 | 124 | import org.apache.beam.sdk.transforms.PTransform; |
120 | 125 | import org.apache.beam.sdk.transforms.ParDo; |
| 126 | +import org.apache.beam.sdk.transforms.Redistribute; |
121 | 127 | import org.apache.beam.sdk.transforms.Reshuffle; |
122 | 128 | import org.apache.beam.sdk.transforms.SerializableFunction; |
123 | 129 | import org.apache.beam.sdk.transforms.SerializableFunctions; |
@@ -669,6 +675,19 @@ public static TypedRead<TableRow> readTableRowsWithSchema() { |
669 | 675 | BigQueryUtils.tableRowToBeamRow(), |
670 | 676 | BigQueryUtils.tableRowFromBeamRow()); |
671 | 677 | } |
| 678 | + /** @deprecated this method may have breaking changes introduced, use with caution */ |
| 679 | + @Deprecated |
| 680 | + public static DynamicRead readDynamicallyTableRows() { |
| 681 | + return new AutoValue_BigQueryIO_DynamicRead.Builder() |
| 682 | + .setBigQueryServices(new BigQueryServicesImpl()) |
| 683 | + .setParseFn(new TableRowParser()) |
| 684 | + .setFormat(DataFormat.AVRO) |
| 685 | + .setOutputCoder(TableRowJsonCoder.of()) |
| 686 | + .setProjectionPushdownApplied(false) |
| 687 | + .setBadRecordErrorHandler(new DefaultErrorHandler<>()) |
| 688 | + .setBadRecordRouter(BadRecordRouter.THROWING_ROUTER) |
| 689 | + .build(); |
| 690 | + } |
672 | 691 |
|
673 | 692 | private static class TableSchemaFunction |
674 | 693 | implements Serializable, Function<@Nullable String, @Nullable TableSchema> { |
@@ -804,6 +823,99 @@ public TableRow apply(SchemaAndRecord schemaAndRecord) { |
804 | 823 | return BigQueryAvroUtils.convertGenericRecordToTableRow(schemaAndRecord.getRecord()); |
805 | 824 | } |
806 | 825 | } |
| 826 | + /** @deprecated this class may have breaking changes introduced, use with caution */ |
| 827 | + @Deprecated |
| 828 | + @AutoValue |
| 829 | + public abstract static class DynamicRead |
| 830 | + extends PTransform<PCollection<BigQueryDynamicReadDescriptor>, PCollection<TableRow>> { |
| 831 | + |
| 832 | + abstract BigQueryServices getBigQueryServices(); |
| 833 | + |
| 834 | + abstract DataFormat getFormat(); |
| 835 | + |
| 836 | + abstract @Nullable SerializableFunction<SchemaAndRecord, TableRow> getParseFn(); |
| 837 | + |
| 838 | + abstract @Nullable ValueProvider<String> getRowRestriction(); |
| 839 | + |
| 840 | + abstract @Nullable Coder<TableRow> getOutputCoder(); |
| 841 | + |
| 842 | + abstract boolean getProjectionPushdownApplied(); |
| 843 | + |
| 844 | + abstract BadRecordRouter getBadRecordRouter(); |
| 845 | + |
| 846 | + abstract ErrorHandler<BadRecord, ?> getBadRecordErrorHandler(); |
| 847 | + |
| 848 | + @AutoValue.Builder |
| 849 | + abstract static class Builder { |
| 850 | + |
| 851 | + abstract Builder setFormat(DataFormat format); |
| 852 | + |
| 853 | + abstract Builder setBigQueryServices(BigQueryServices bigQueryServices); |
| 854 | + |
| 855 | + abstract Builder setParseFn(SerializableFunction<SchemaAndRecord, TableRow> parseFn); |
| 856 | + |
| 857 | + abstract Builder setRowRestriction(ValueProvider<String> rowRestriction); |
| 858 | + |
| 859 | + abstract Builder setOutputCoder(Coder<TableRow> coder); |
| 860 | + |
| 861 | + abstract Builder setProjectionPushdownApplied(boolean projectionPushdownApplied); |
| 862 | + |
| 863 | + abstract Builder setBadRecordErrorHandler(ErrorHandler<BadRecord, ?> badRecordErrorHandler); |
| 864 | + |
| 865 | + abstract Builder setBadRecordRouter(BadRecordRouter badRecordRouter); |
| 866 | + |
| 867 | + abstract DynamicRead build(); |
| 868 | + } |
| 869 | + |
| 870 | + DynamicRead() {} |
| 871 | + |
| 872 | + class CreateBoundedSourceForTable |
| 873 | + extends DoFn<BigQueryDynamicReadDescriptor, BigQueryStorageStreamSource<TableRow>> { |
| 874 | + |
| 875 | + @ProcessElement |
| 876 | + public void processElement( |
| 877 | + OutputReceiver<BigQueryStorageStreamSource<TableRow>> receiver, |
| 878 | + @Element BigQueryDynamicReadDescriptor descriptor, |
| 879 | + PipelineOptions options) |
| 880 | + throws Exception { |
| 881 | + BigQueryStorageTableSource<TableRow> output = |
| 882 | + BigQueryStorageTableSource.create( |
| 883 | + StaticValueProvider.of(BigQueryHelpers.parseTableSpec(descriptor.getTable())), |
| 884 | + getFormat(), |
| 885 | + null, |
| 886 | + getRowRestriction(), |
| 887 | + getParseFn(), |
| 888 | + getOutputCoder(), |
| 889 | + getBigQueryServices(), |
| 890 | + getProjectionPushdownApplied()); |
| 891 | + // 1mb --> 1 shard; 1gb --> 32 shards; 1tb --> 1000 shards, 1pb --> 32k |
| 892 | + // shards |
| 893 | + long desiredChunkSize = |
| 894 | + Math.max(1 << 20, (long) (1000 * Math.sqrt(output.getEstimatedSizeBytes(options)))); |
| 895 | + List<BigQueryStorageStreamSource<TableRow>> split = output.split(desiredChunkSize, options); |
| 896 | + split.stream().forEach(source -> receiver.output(source)); |
| 897 | + } |
| 898 | + } |
| 899 | + |
| 900 | + @Override |
| 901 | + public PCollection<TableRow> expand(PCollection<BigQueryDynamicReadDescriptor> input) { |
| 902 | + TupleTag<TableRow> rowTag = new TupleTag<>(); |
| 903 | + PCollectionTuple resultTuple = |
| 904 | + input |
| 905 | + .apply("convert", ParDo.of(new CreateBoundedSourceForTable())) |
| 906 | + .apply("redistribute", Redistribute.arbitrarily()) |
| 907 | + .apply( |
| 908 | + "Read Storage Table Source", |
| 909 | + ParDo.of( |
| 910 | + new TypedRead.ReadTableSource<TableRow>( |
| 911 | + rowTag, getParseFn(), getBadRecordRouter())) |
| 912 | + .withOutputTags(rowTag, TupleTagList.of(BAD_RECORD_TAG))); |
| 913 | + getBadRecordErrorHandler() |
| 914 | + .addErrorCollection( |
| 915 | + resultTuple.get(BAD_RECORD_TAG).setCoder(BadRecord.getCoder(input.getPipeline()))); |
| 916 | + return resultTuple.get(rowTag).setCoder(TableRowJsonCoder.of()); |
| 917 | + } |
| 918 | + } |
807 | 919 |
|
808 | 920 | /** Implementation of {@link BigQueryIO#read()}. */ |
809 | 921 | public static class Read extends PTransform<PBegin, PCollection<TableRow>> { |
|
0 commit comments