forked from GoogleCloudPlatform/DataflowTemplates
/
TextImportPipeline.java
321 lines (280 loc) · 13.5 KB
/
TextImportPipeline.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
/*
* 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.spanner;
import com.google.cloud.spanner.Options.RpcPriority;
import com.google.cloud.spanner.SpannerOptions;
import com.google.cloud.teleport.metadata.Template;
import com.google.cloud.teleport.metadata.TemplateCategory;
import com.google.cloud.teleport.metadata.TemplateCreationParameter;
import com.google.cloud.teleport.metadata.TemplateParameter;
import com.google.cloud.teleport.metadata.TemplateParameter.TemplateEnumOption;
import com.google.cloud.teleport.spanner.TextImportPipeline.Options;
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
import org.apache.beam.sdk.transforms.SerializableFunction;
/**
* Text files to Cloud Spanner Import pipeline. This pipeline ingests CSV and other type of
* delimited data from GCS and writes data to a Cloud Spanner database table. Each row from the
* input CSV file will be applied to Cloud Spanner with an InsertOrUpdate mutation, so this can be
* used both to populate new rows or to update columns of existing rows.
*
* <p>You can specify column delimiter other than comma. Also make sure to use field qualifier such
* as double quote to escape delimiter if it is in the value.
*
* <p>Text file must NOT have a header.
*
* <p>Example Usage: Here is CSV sample data simulating an account table:
* 1,sample_user_1,true,2018-01-01,2018-01-01T12:30:00Z
*
* <p>Schema file must have all column and type definition in one line. Schema file must use the
* data type names of Cloud Spanner. We currently support the following Cloud Spanner data types: -
* BOOL - DATE - FLOAT64 - INT64 - STRING - TIMESTAMP
*
* <p>Input format properties: - \\N in the source column will be considered as NULL value when
* writing to Cloud Spanner. - If you need to escape characters, you can use the "fieldQualifier"
* parameter to tell the pipeline. e.g. You can put all values inside double quotes like "123",
* "john", "true" - See the implementation of parseRow() below to see what values are accepted for
* each data type.
*
* <p>NOTE: BYTES, ARRAY, STRUCT types are not supported.
*
* <p>Example schema file for the CSV file above:
*
* <pre>Id:INT64,Username:STRING,Active:BOOL,CreateDate:DATE,ModifyTime:TIMESTAMP</pre>
*
* <p>Here is the DDL for creating Cloud Spanner table:
*
* <pre>CREATE TABLE example_table
* ( Id INT64, Username STRING(MAX), Active BOOL, CreateDate DATE, ModifyTime TIMESTAMP )
* PRIMARY KEY(Id)
* </pre>
*
* <p>Check out <a
* href="https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/main/v1/README_GCS_Text_to_Cloud_Spanner.md">README</a>
* for instructions on how to use or modify this template.
*/
@Template(
name = "GCS_Text_to_Cloud_Spanner",
category = TemplateCategory.BATCH,
displayName = "Text Files on Cloud Storage to Cloud Spanner",
description =
"A pipeline to import a Cloud Spanner database from a set of Text (CSV) files in Cloud"
+ " Storage.",
optionsClass = Options.class,
documentation =
"https://cloud.google.com/dataflow/docs/guides/templates/provided/cloud-storage-to-cloud-spanner",
contactInformation = "https://cloud.google.com/support")
public class TextImportPipeline {
/** Options for {@link TextImportPipeline}. */
public interface Options extends PipelineOptions {
@TemplateParameter.Text(
order = 1,
regexes = {"^[a-z0-9\\-]+$"},
description = "Cloud Spanner instance id",
helpText = "The instance id of the Cloud Spanner database that you want to import to.")
ValueProvider<String> getInstanceId();
void setInstanceId(ValueProvider<String> value);
@TemplateParameter.Text(
order = 2,
regexes = {"^[a-z_0-9\\-]+$"},
description = "Cloud Spanner database id",
helpText =
"The database id of the Cloud Spanner database that you want to import into (must"
+ " already exist, and with the destination tables created).")
ValueProvider<String> getDatabaseId();
void setDatabaseId(ValueProvider<String> value);
@TemplateParameter.Text(
order = 3,
optional = true,
description = "Cloud Spanner Endpoint to call",
helpText = "The Cloud Spanner endpoint to call in the template. Only used for testing.",
example = "https://batch-spanner.googleapis.com")
@Default.String("https://batch-spanner.googleapis.com")
ValueProvider<String> getSpannerHost();
void setSpannerHost(ValueProvider<String> value);
@TemplateParameter.GcsReadFile(
order = 4,
description = "Text Import Manifest file",
helpText =
"The Cloud Storage path and filename of the text import manifest file. Text Import"
+ " Manifest file, storing a json-encoded importManifest object.",
example = "gs://your-bucket/your-folder/your-manifest.json")
ValueProvider<String> getImportManifest();
void setImportManifest(ValueProvider<String> value);
@TemplateParameter.Text(
order = 5,
optional = true,
description = "Column delimiter of the data files",
helpText = "The column delimiter of the input text files. Defaults to ','",
example = ",")
@Default.Character(',')
ValueProvider<Character> getColumnDelimiter();
void setColumnDelimiter(ValueProvider<Character> value);
@TemplateParameter.Text(
order = 6,
optional = true,
description = "Field qualifier used by the source file",
helpText =
"The field qualifier used by the source file. It should be used when character needs to"
+ " be escaped. Field qualifier should be used when character needs to be escaped."
+ " The default value is double quotes.")
@Default.Character('"')
ValueProvider<Character> getFieldQualifier();
void setFieldQualifier(ValueProvider<Character> value);
@TemplateParameter.Boolean(
order = 7,
optional = true,
description = "If true, the lines has trailing delimiters",
helpText =
"The flag indicating whether or not the input lines have trailing delimiters. The"
+ " default value is true. If the text file contains trailing delimiter, then set"
+ " trailingDelimiter parameter to true during pipeline execution to import a Cloud"
+ " Spanner database from a set of text files, otherwise set it to false.")
@Default.Boolean(true)
ValueProvider<Boolean> getTrailingDelimiter();
void setTrailingDelimiter(ValueProvider<Boolean> value);
@TemplateParameter.Text(
order = 8,
optional = true,
description = "Escape character",
helpText =
"The escape character. The default value is NULL (not using the escape character).")
ValueProvider<Character> getEscape();
void setEscape(ValueProvider<Character> value);
@TemplateParameter.Text(
order = 9,
optional = true,
description = "Null String",
helpText =
"The string that represents the NULL value. The default value is null (not using the"
+ " null string).")
ValueProvider<String> getNullString();
void setNullString(ValueProvider<String> value);
@TemplateParameter.Text(
order = 10,
optional = true,
description = "Date format",
helpText =
"The format used to parse date columns. By default, the pipeline tries to parse the"
+ " date columns as \"yyyy-MM-dd[' 00:00:00']\" (e.g., 2019-01-31, or 2019-01-31"
+ " 00:00:00). If your data format is different, please specify the format using"
+ " the java.time.format.DateTimeFormatter patterns. For more details, please refer"
+ " to https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/time/format/DateTimeFormatter.html")
ValueProvider<String> getDateFormat();
void setDateFormat(ValueProvider<String> value);
@TemplateParameter.Text(
order = 11,
optional = true,
description = "Timestamp format",
helpText =
"The format used to parse timestamp columns. If the timestamp is a long integer, then"
+ " it is treated as Unix epoch (the microsecond since 1970-01-01T00:00:00.000Z."
+ " Otherwise, it is parsed as a string using the"
+ " java.time.format.DateTimeFormatter.ISO_INSTANT format. For other cases, please"
+ " specify you own pattern string, e.g., \"MMM dd yyyy HH:mm:ss.SSSVV\" for"
+ " timestamp in the form of \"Jan 21 1998 01:02:03.456+08:00\". For more details,"
+ " please refer to"
+ " https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/time/format/DateTimeFormatter.html")
ValueProvider<String> getTimestampFormat();
void setTimestampFormat(ValueProvider<String> value);
@TemplateCreationParameter(value = "false")
@Description("If true, wait for job finish. The default value is true.")
@Default.Boolean(true)
boolean getWaitUntilFinish();
void setWaitUntilFinish(boolean value);
@TemplateParameter.ProjectId(
order = 13,
optional = true,
description = "Cloud Spanner Project Id",
helpText = "The project id of the Cloud Spanner instance.")
ValueProvider<String> getSpannerProjectId();
void setSpannerProjectId(ValueProvider<String> value);
@TemplateParameter.Enum(
order = 14,
enumOptions = {
@TemplateEnumOption("LOW"),
@TemplateEnumOption("MEDIUM"),
@TemplateEnumOption("HIGH")
},
optional = true,
description = "Priority for Spanner RPC invocations",
helpText =
"The request priority for Cloud Spanner calls. The value must be one of:"
+ " [HIGH,MEDIUM,LOW].")
ValueProvider<RpcPriority> getSpannerPriority();
void setSpannerPriority(ValueProvider<RpcPriority> value);
@TemplateParameter.Boolean(
order = 15,
optional = true,
description = "Handle new line",
helpText =
"If true, run the template in handleNewLine mode, which is slower but handles newline"
+ " characters inside data.")
@Default.Boolean(false)
ValueProvider<Boolean> getHandleNewLine();
void setHandleNewLine(ValueProvider<Boolean> value);
@TemplateParameter.GcsWriteFolder(
order = 16,
description = "Invalid rows output path",
optional = true,
helpText = "Cloud Storage path where to write rows that cannot be imported.",
example = "gs://your-bucket/your-path")
@Default.String("")
ValueProvider<String> getInvalidOutputPath();
void setInvalidOutputPath(ValueProvider<String> value);
}
public static void main(String[] args) {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
Pipeline p = Pipeline.create(options);
SpannerConfig spannerConfig =
SpannerConfig.create()
// Temporary fix explicitly setting SpannerConfig.projectId to the default project
// if spannerProjectId is not provided as a parameter. Required as of Beam 2.38,
// which no longer accepts null label values on metrics, and SpannerIO#setup() has
// a bug resulting in the label value being set to the original parameter value,
// with no fallback to the default project.
// TODO: remove NestedValueProvider when this is fixed in Beam.
.withProjectId(
NestedValueProvider.of(
options.getSpannerProjectId(),
(SerializableFunction<String, String>)
input -> input != null ? input : SpannerOptions.getDefaultProjectId()))
.withHost(options.getSpannerHost())
.withInstanceId(options.getInstanceId())
.withDatabaseId(options.getDatabaseId())
.withRpcPriority(options.getSpannerPriority());
p.apply(
new TextImportTransform(
spannerConfig, options.getImportManifest(), options.getInvalidOutputPath()));
PipelineResult result = p.run();
if (options.getWaitUntilFinish()
&&
/* Only if template location is null, there is a dataflow job to wait for. Otherwise it's
* template generation, which doesn't start a dataflow job.
*/
options.as(DataflowPipelineOptions.class).getTemplateLocation() == null) {
result.waitUntilFinish();
}
}
}