forked from GoogleCloudPlatform/DataflowTemplates
-
Notifications
You must be signed in to change notification settings - Fork 0
/
BigQueryToTFRecord.java
334 lines (305 loc) · 12.3 KB
/
BigQueryToTFRecord.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
/*
* Copyright (C) 2019 Google LLC
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package com.google.cloud.teleport.templates;
import com.google.api.services.bigquery.model.TableFieldSchema;
import com.google.cloud.teleport.templates.common.BigQueryConverters.BigQueryReadOptions;
import com.google.protobuf.ByteString;
import java.util.Iterator;
import java.util.Random;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.util.Utf8;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.io.FileIO;
import org.apache.beam.sdk.io.TFRecordIO;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.transforms.Partition;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.tensorflow.example.Example;
import org.tensorflow.example.Feature;
import org.tensorflow.example.Features;
/**
* Dataflow template which reads BigQuery data and writes it to GCS as a set of TFRecords. The
* source is a SQL query.
*/
public class BigQueryToTFRecord {
/**
* The {@link BigQueryToTFRecord#buildFeatureFromIterator(Class, Object, Feature.Builder)} method
* handles {@link GenericData.Array} that are passed into the {@link
* BigQueryToTFRecord#buildFeature} method creating a TensorFlow feature from the record.
*/
private static final String TRAIN = "train/";
private static final String TEST = "test/";
private static final String VAL = "val/";
private static void buildFeatureFromIterator(
Class<?> fieldType, Object field, Feature.Builder feature) {
ByteString byteString;
GenericData.Array f = (GenericData.Array) field;
if (fieldType == Long.class) {
Iterator<Long> longIterator = f.iterator();
while (longIterator.hasNext()) {
Long longValue = longIterator.next();
feature.getInt64ListBuilder().addValue(longValue);
}
} else if (fieldType == double.class) {
Iterator<Double> doubleIterator = f.iterator();
while (doubleIterator.hasNext()) {
double doubleValue = doubleIterator.next();
feature.getFloatListBuilder().addValue((float) doubleValue);
}
} else if (fieldType == String.class) {
Iterator<Utf8> stringIterator = f.iterator();
while (stringIterator.hasNext()) {
String stringValue = stringIterator.next().toString();
byteString = ByteString.copyFromUtf8(stringValue);
feature.getBytesListBuilder().addValue(byteString);
}
} else if (fieldType == boolean.class) {
Iterator<Boolean> booleanIterator = f.iterator();
while (booleanIterator.hasNext()) {
Boolean boolValue = booleanIterator.next();
int boolAsInt = boolValue ? 1 : 0;
feature.getInt64ListBuilder().addValue(boolAsInt);
}
}
}
/**
* The {@link BigQueryToTFRecord#buildFeature} method takes in an individual field and type
* corresponding to a column value from a SchemaAndRecord Object returned from a BigQueryIO.read()
* step. The method builds a TensorFlow Feature based on the type of the object- ie: STRING, TIME,
* INTEGER etc..
*/
private static Feature buildFeature(Object field, String type) {
Feature.Builder feature = Feature.newBuilder();
ByteString byteString;
switch (type) {
case "STRING":
case "TIME":
case "DATE":
if (field instanceof GenericData.Array) {
buildFeatureFromIterator(String.class, field, feature);
} else {
byteString = ByteString.copyFromUtf8(field.toString());
feature.getBytesListBuilder().addValue(byteString);
}
break;
case "BYTES":
byteString = ByteString.copyFrom((byte[]) field);
feature.getBytesListBuilder().addValue(byteString);
break;
case "INTEGER":
case "INT64":
case "TIMESTAMP":
if (field instanceof GenericData.Array) {
buildFeatureFromIterator(Long.class, field, feature);
} else {
feature.getInt64ListBuilder().addValue((long) field);
}
break;
case "FLOAT":
case "FLOAT64":
if (field instanceof GenericData.Array) {
buildFeatureFromIterator(double.class, field, feature);
} else {
feature.getFloatListBuilder().addValue((float) (double) field);
}
break;
case "BOOLEAN":
case "BOOL":
if (field instanceof GenericData.Array) {
buildFeatureFromIterator(boolean.class, field, feature);
} else {
int boolAsInt = (boolean) field ? 1 : 0;
feature.getInt64ListBuilder().addValue(boolAsInt);
}
break;
default:
throw new RuntimeException("Unsupported type: " + type);
}
return feature.build();
}
/**
* The {@link BigQueryToTFRecord#record2Example(SchemaAndRecord)} method uses takes in a
* SchemaAndRecord Object returned from a BigQueryIO.read() step and builds a TensorFlow Example
* from the record.
*/
@VisibleForTesting
protected static byte[] record2Example(SchemaAndRecord schemaAndRecord) {
Example.Builder example = Example.newBuilder();
Features.Builder features = example.getFeaturesBuilder();
GenericRecord record = schemaAndRecord.getRecord();
for (TableFieldSchema field : schemaAndRecord.getTableSchema().getFields()) {
Object fieldValue = record.get(field.getName());
if (fieldValue != null) {
Feature feature = buildFeature(fieldValue, field.getType());
features.putFeature(field.getName(), feature);
}
}
return example.build().toByteArray();
}
/**
* The {@link BigQueryToTFRecord#concatURI} method uses takes in a Cloud Storage URI and a
* subdirectory name and safely concatenates them. The resulting String is used as a sink for
* TFRecords.
*/
private static String concatURI(String dir, String folder) {
if (dir.endsWith("/")) {
return dir + folder;
} else {
return dir + "/" + folder;
}
}
/**
* The {@link BigQueryToTFRecord#applyTrainTestValSplit} method transforms the PCollection by
* randomly partitioning it into PCollections for each dataset.
*/
static PCollectionList<byte[]> applyTrainTestValSplit(
PCollection<byte[]> input,
ValueProvider<Float> trainingPercentage,
ValueProvider<Float> testingPercentage,
ValueProvider<Float> validationPercentage,
Random rand) {
return input.apply(
Partition.of(
3,
(Partition.PartitionFn<byte[]>)
(number, numPartitions) -> {
Float train = trainingPercentage.get();
Float test = testingPercentage.get();
Float validation = validationPercentage.get();
Double d = rand.nextDouble();
if (train + test + validation != 1) {
throw new RuntimeException(
String.format(
"Train %.2f, Test %.2f, Validation"
+ " %.2f percentages must add up to 100 percent",
train, test, validation));
}
if (d < train) {
return 0;
} else if (d >= train && d < train + test) {
return 1;
} else {
return 2;
}
}));
}
/** Run the pipeline. */
public static void main(String[] args) {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
run(options);
}
/**
* Runs the pipeline to completion with the specified options. This method does not wait until the
* pipeline is finished before returning. Invoke {@code result.waitUntilFinish()} on the result
* object to block until the pipeline is finished running if blocking programmatic execution is
* required.
*
* @param options The execution options.
* @return The pipeline result.
*/
public static PipelineResult run(Options options) {
Random rand = new Random(100); // set random seed
Pipeline pipeline = Pipeline.create(options);
PCollection<byte[]> bigQueryToExamples =
pipeline
.apply(
"RecordToExample",
BigQueryIO.read(BigQueryToTFRecord::record2Example)
.fromQuery(options.getReadQuery())
.withCoder(ByteArrayCoder.of())
.withTemplateCompatibility()
.withoutValidation()
.usingStandardSql()
.withMethod(BigQueryIO.TypedRead.Method.DIRECT_READ)
// Enable BigQuery Storage API
)
.apply("ReshuffleResults", Reshuffle.viaRandomKey());
PCollectionList<byte[]> partitionedExamples =
applyTrainTestValSplit(
bigQueryToExamples,
options.getTrainingPercentage(),
options.getTestingPercentage(),
options.getValidationPercentage(),
rand);
partitionedExamples
.get(0)
.apply(
"WriteTFTrainingRecord",
FileIO.<byte[]>write()
.via(TFRecordIO.sink())
.to(
ValueProvider.NestedValueProvider.of(
options.getOutputDirectory(), dir -> concatURI(dir, TRAIN)))
.withNumShards(0)
.withSuffix(options.getOutputSuffix()));
partitionedExamples
.get(1)
.apply(
"WriteTFTestingRecord",
FileIO.<byte[]>write()
.via(TFRecordIO.sink())
.to(
ValueProvider.NestedValueProvider.of(
options.getOutputDirectory(), dir -> concatURI(dir, TEST)))
.withNumShards(0)
.withSuffix(options.getOutputSuffix()));
partitionedExamples
.get(2)
.apply(
"WriteTFValidationRecord",
FileIO.<byte[]>write()
.via(TFRecordIO.sink())
.to(
ValueProvider.NestedValueProvider.of(
options.getOutputDirectory(), dir -> concatURI(dir, VAL)))
.withNumShards(0)
.withSuffix(options.getOutputSuffix()));
return pipeline.run();
}
/** Define command line arguments. */
public interface Options extends BigQueryReadOptions {
@Description("The GCS directory to store output TFRecord files.")
ValueProvider<String> getOutputDirectory();
void setOutputDirectory(ValueProvider<String> outputDirectory);
@Description("The output suffix for TFRecord Files")
@Default.String(".tfrecord")
ValueProvider<String> getOutputSuffix();
void setOutputSuffix(ValueProvider<String> outputSuffix);
@Description("The training percentage split for TFRecord Files")
@Default.Float(1)
ValueProvider<Float> getTrainingPercentage();
void setTrainingPercentage(ValueProvider<Float> trainingPercentage);
@Description("The testing percentage split for TFRecord Files")
@Default.Float(0)
ValueProvider<Float> getTestingPercentage();
void setTestingPercentage(ValueProvider<Float> testingPercentage);
@Description("The validation percentage split for TFRecord Files")
@Default.Float(0)
ValueProvider<Float> getValidationPercentage();
void setValidationPercentage(ValueProvider<Float> validationPercentage);
}
}