forked from wepay/kafka-connect-bigquery
-
Notifications
You must be signed in to change notification settings - Fork 105
/
MergeQueries.java
510 lines (469 loc) · 21.8 KB
/
MergeQueries.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
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
/*
* Copyright 2020 Confluent, Inc.
*
* This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc.
*
* 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.wepay.kafka.connect.bigquery;
import com.google.cloud.bigquery.BigQuery;
import com.google.cloud.bigquery.BigQueryException;
import com.google.cloud.bigquery.Field;
import com.google.cloud.bigquery.FieldList;
import com.google.cloud.bigquery.QueryJobConfiguration;
import com.google.cloud.bigquery.Schema;
import com.google.cloud.bigquery.TableId;
import com.google.common.annotations.VisibleForTesting;
import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig;
import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig;
import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException;
import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException;
import com.wepay.kafka.connect.bigquery.utils.SleepUtils;
import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor;
import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches;
import com.wepay.kafka.connect.bigquery.write.row.BigQueryErrorResponses;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.sink.SinkTaskContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.destTable;
import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.intTable;
public class MergeQueries {
private static final int WAIT_MAX_JITTER = 1000;
public static final String INTERMEDIATE_TABLE_KEY_FIELD_NAME = "key";
public static final String INTERMEDIATE_TABLE_VALUE_FIELD_NAME = "value";
public static final String INTERMEDIATE_TABLE_ITERATION_FIELD_NAME = "i";
public static final String INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME = "partitionTime";
public static final String INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD = "batchNumber";
public static final String DESTINATION_TABLE_ALIAS = "dstTableAlias";
private static final Logger logger = LoggerFactory.getLogger(MergeQueries.class);
private final String keyFieldName;
private final boolean insertPartitionTime;
private final boolean upsertEnabled;
private final boolean deleteEnabled;
private final int bigQueryRetry;
private final long bigQueryRetryWait;
private final MergeBatches mergeBatches;
private final KCBQThreadPoolExecutor executor;
private final BigQuery bigQuery;
private final SchemaManager schemaManager;
private final SinkTaskContext context;
public MergeQueries(BigQuerySinkTaskConfig config,
MergeBatches mergeBatches,
KCBQThreadPoolExecutor executor,
BigQuery bigQuery,
SchemaManager schemaManager,
SinkTaskContext context) {
this(
config.getKafkaKeyFieldName().orElseThrow(() ->
new ConnectException("Kafka key field must be configured when upsert/delete is enabled")
),
config.getBoolean(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG),
config.getBoolean(BigQuerySinkConfig.UPSERT_ENABLED_CONFIG),
config.getBoolean(BigQuerySinkConfig.DELETE_ENABLED_CONFIG),
config.getInt(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG),
config.getLong(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG),
mergeBatches,
executor,
bigQuery,
schemaManager,
context
);
}
@VisibleForTesting
MergeQueries(String keyFieldName,
boolean insertPartitionTime,
boolean upsertEnabled,
boolean deleteEnabled,
int bigQueryRetry,
long bigQueryRetryWait,
MergeBatches mergeBatches,
KCBQThreadPoolExecutor executor,
BigQuery bigQuery,
SchemaManager schemaManager,
SinkTaskContext context) {
this.keyFieldName = keyFieldName;
this.insertPartitionTime = insertPartitionTime;
this.upsertEnabled = upsertEnabled;
this.deleteEnabled = deleteEnabled;
this.bigQueryRetry = bigQueryRetry;
this.bigQueryRetryWait = bigQueryRetryWait;
this.mergeBatches = mergeBatches;
this.executor = executor;
this.bigQuery = bigQuery;
this.schemaManager = schemaManager;
this.context = context;
}
public void mergeFlushAll() {
logger.debug("Triggering merge flush for all tables");
mergeBatches.intermediateTables().forEach(this::mergeFlush);
}
public void mergeFlush(TableId intermediateTable) {
if(mergeBatches.isCurrentBatchEmpty(intermediateTable)){
logger.debug("Merge flush is not performed as the current batch is empty.");
return;
}
final TableId destinationTable = mergeBatches.destinationTableFor(intermediateTable);
final int batchNumber = mergeBatches.incrementBatch(intermediateTable);
logger.trace("Triggering merge flush from {} to {} for batch {}",
intTable(intermediateTable), destTable(destinationTable), batchNumber);
executor.execute(() -> {
try {
mergeFlush(intermediateTable, destinationTable, batchNumber);
} catch (InterruptedException e) {
throw new ExpectedInterruptException(String.format(
"Interrupted while performing merge flush of batch %d from %s to %s",
batchNumber, intTable(intermediateTable), destTable(destinationTable)));
}
});
}
private void mergeFlush(
TableId intermediateTable, TableId destinationTable, int batchNumber
) throws InterruptedException{
// If there are rows to flush in this batch, flush them
if (mergeBatches.prepareToFlush(intermediateTable, batchNumber)) {
logger.debug("Running merge query on batch {} from {}",
batchNumber, intTable(intermediateTable));
String mergeFlushQuery = mergeFlushQuery(intermediateTable, destinationTable, batchNumber);
logger.trace(mergeFlushQuery);
int attempt = 0;
boolean success = false;
while (!success) {
try {
if (attempt > 0) {
SleepUtils.waitRandomTime(this.bigQueryRetryWait, WAIT_MAX_JITTER);
}
bigQuery.query(QueryJobConfiguration.of(mergeFlushQuery));
success = true;
} catch (BigQueryException e) {
if (attempt >= bigQueryRetry) {
throw new BigQueryConnectException("Failed to merge rows to destination table `" + destinationTable + "` within " + attempt
+ " attempts.", e);
} else if (BigQueryErrorResponses.isCouldNotSerializeAccessError(e)) {
logger.warn("Serialization error while merging from {} to {}, attempting retry", intermediateTable, destinationTable);
attempt++;
} else if (BigQueryErrorResponses.isJobInternalError(e)) {
logger.warn("Job internal error while merging from {} to {}, attempting retry", intermediateTable, destinationTable);
attempt++;
} else {
throw e;
}
}
}
logger.trace("Merge from {} to {} completed",
intTable(intermediateTable), destTable(destinationTable));
logger.debug("Recording flush success for batch {} from {}",
batchNumber, intTable(intermediateTable));
mergeBatches.recordSuccessfulFlush(intermediateTable, batchNumber);
// Commit those offsets ASAP
context.requestCommit();
logger.info("Completed merge flush of batch {} from {} to {}",
batchNumber, intTable(intermediateTable), destTable(destinationTable));
}
// After, regardless of whether we flushed or not, clean up old batches from the intermediate
// table. Some rows may be several batches old but still in the table if they were in the
// streaming buffer during the last purge.
logger.trace("Clearing batches from {} on back from {}", batchNumber, intTable(intermediateTable));
String batchClearQuery = batchClearQuery(intermediateTable, batchNumber);
logger.trace(batchClearQuery);
bigQuery.query(QueryJobConfiguration.of(batchClearQuery));
}
@VisibleForTesting
String mergeFlushQuery(TableId intermediateTable, TableId destinationTable, int batchNumber) {
Schema intermediateSchema = schemaManager.cachedSchema(intermediateTable);
if (upsertEnabled && deleteEnabled) {
return upsertDeleteMergeFlushQuery(intermediateTable, destinationTable, batchNumber, intermediateSchema);
} else if (upsertEnabled) {
return upsertMergeFlushQuery(intermediateTable, destinationTable, batchNumber, intermediateSchema);
} else if (deleteEnabled) {
return deleteMergeFlushQuery(intermediateTable, destinationTable, batchNumber, intermediateSchema);
} else {
throw new IllegalStateException("At least one of upsert or delete must be enabled for merge flushing to occur.");
}
}
/*
MERGE `<dataset>`.`<destinationTable>`
USING (
SELECT * FROM (
SELECT ARRAY_AGG(
x ORDER BY i DESC LIMIT 1
)[OFFSET(0)] src
FROM `<dataset>`.`<intermediateTable>` x
WHERE batchNumber=<batchNumber>
GROUP BY key.<field>[, key.<field>...]
)
)
ON `<destinationTable>`.<keyField>=src.key
WHEN MATCHED AND src.value IS NOT NULL
THEN UPDATE SET `<valueField>`=src.value.<field>[, `<valueField>`=src.value.<field>...]
WHEN MATCHED AND src.value IS NULL
THEN DELETE
WHEN NOT MATCHED AND src.value IS NOT NULL
THEN INSERT (`<keyField>`, [_PARTITIONTIME, ]`<valueField>`[, `<valueField>`])
VALUES (
src.key,
[CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),]
src.value.<field>[, src.value.<field>...]
);
*/
private String upsertDeleteMergeFlushQuery(
TableId intermediateTable, TableId destinationTable, int batchNumber, Schema intermediateSchema
) {
List<String> keyFields = listFields(
intermediateSchema.getFields().get(INTERMEDIATE_TABLE_KEY_FIELD_NAME).getSubFields(),
INTERMEDIATE_TABLE_KEY_FIELD_NAME + "."
);
List<String> valueColumns = valueColumns(intermediateSchema);
final String key = INTERMEDIATE_TABLE_KEY_FIELD_NAME;
final String i = INTERMEDIATE_TABLE_ITERATION_FIELD_NAME;
final String value = INTERMEDIATE_TABLE_VALUE_FIELD_NAME;
final String batch = INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD;
return "MERGE " + table(destinationTable) + " " + DESTINATION_TABLE_ALIAS + " "
+ "USING ("
+ "SELECT * FROM ("
+ "SELECT ARRAY_AGG("
+ "x ORDER BY " + i + " DESC LIMIT 1"
+ ")[OFFSET(0)] src "
+ "FROM " + table(intermediateTable) + " x "
+ "WHERE " + batch + "=" + batchNumber + " "
+ "GROUP BY " + String.join(", ", keyFields)
+ ")"
+ ") "
+ "ON " + DESTINATION_TABLE_ALIAS + "." + keyFieldName + "=src." + key + " "
+ "WHEN MATCHED AND src." + value + " IS NOT NULL "
+ "THEN UPDATE SET " + valueColumns.stream().map(col -> DESTINATION_TABLE_ALIAS + ".`" + col + "`=src." + value + "." + col).collect(Collectors.joining(", ")) + " "
+ "WHEN MATCHED AND src." + value + " IS NULL "
+ "THEN DELETE "
+ "WHEN NOT MATCHED AND src." + value + " IS NOT NULL "
+ "THEN INSERT (`"
+ keyFieldName + "`, "
+ partitionTimePseudoColumn()
+ "`"
+ String.join("`, `", valueColumns) + "`) "
+ "VALUES ("
+ "src." + key + ", "
+ partitionTimeValue()
+ valueColumns.stream().map(col -> "src." + value + "." + col).collect(Collectors.joining(", "))
+ ");";
}
/*
MERGE `<dataset>`.`<destinationTable>`
USING (
SELECT * FROM (
SELECT ARRAY_AGG(
x ORDER BY i DESC LIMIT 1
)[OFFSET(0)] src
FROM `<dataset>`.`<intermediateTable>` x
WHERE batchNumber=<batchNumber>
GROUP BY key.<field>[, key.<field>...]
)
)
ON `<destinationTable>`.<keyField>=src.key
WHEN MATCHED
THEN UPDATE SET `<valueField>`=src.value.<field>[, `<valueField>`=src.value.<field>...]
WHEN NOT MATCHED
THEN INSERT (`<keyField>`, [_PARTITIONTIME, ]`<valueField>`[, `<valueField>`])
VALUES (
src.key,
[CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),]
src.value.<field>[, src.value.<field>...]
);
*/
private String upsertMergeFlushQuery(
TableId intermediateTable, TableId destinationTable, int batchNumber, Schema intermediateSchema
) {
List<String> keyFields = listFields(
intermediateSchema.getFields().get(INTERMEDIATE_TABLE_KEY_FIELD_NAME).getSubFields(),
INTERMEDIATE_TABLE_KEY_FIELD_NAME + "."
);
List<String> valueColumns = valueColumns(intermediateSchema);
final String key = INTERMEDIATE_TABLE_KEY_FIELD_NAME;
final String i = INTERMEDIATE_TABLE_ITERATION_FIELD_NAME;
final String value = INTERMEDIATE_TABLE_VALUE_FIELD_NAME;
final String batch = INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD;
return "MERGE " + table(destinationTable) + " " + DESTINATION_TABLE_ALIAS + " "
+ "USING ("
+ "SELECT * FROM ("
+ "SELECT ARRAY_AGG("
+ "x ORDER BY " + i + " DESC LIMIT 1"
+ ")[OFFSET(0)] src "
+ "FROM " + table(intermediateTable) + " x "
+ "WHERE " + batch + "=" + batchNumber + " "
+ "GROUP BY " + String.join(", ", keyFields)
+ ")"
+ ") "
+ "ON " + DESTINATION_TABLE_ALIAS + "." + keyFieldName + "=src." + key + " "
+ "WHEN MATCHED "
+ "THEN UPDATE SET " + valueColumns.stream().map(col -> DESTINATION_TABLE_ALIAS + ".`" + col + "`=src." + value + "." + col).collect(Collectors.joining(", ")) + " "
+ "WHEN NOT MATCHED "
+ "THEN INSERT (`"
+ keyFieldName + "`, "
+ partitionTimePseudoColumn()
+ "`"
+ String.join("`, `", valueColumns) + "`) "
+ "VALUES ("
+ "src." + key + ", "
+ partitionTimeValue()
+ valueColumns.stream().map(col -> "src." + value + "." + col).collect(Collectors.joining(", "))
+ ");";
}
/*
Delete-only is the trickiest mode. Naively, we could just run a MERGE using the intermediate
table as a source and sort in ascending order of iteration. However, this would miss an edge
case where, for a given key, a non-tombstone record is sent and then followed by a tombstone,
and would result in all rows with that key being deleted from the table, followed by an
insertion of a row for the initial non-tombstone record. This is incorrect; any and all
records with a given key that precede a tombstone should either never make it into BigQuery or
be deleted once the tombstone record is merge flushed.
So instead, we have to try to filter out rows from the source (i.e., intermediate) table
that precede tombstone records for their keys. We do this by:
- Finding the latest tombstone row for each key in the current batch and extracting the
iteration number for each, referring to this as the "deletes" table
- Joining that with the current batch from the intermediate table on the row key, keeping
both tables' iteration numbers (a RIGHT JOIN is used so that rows whose keys don't have
any tombstones present are included with a NULL iteration number for the "deletes" table)
- Filtering out all rows where the "delete" table's iteration number is non-null, and their
iteration number is less than the "delete" table's iteration number
This gives us only rows from the most recent tombstone onward, and works in both cases where
the most recent row for a key is or is not a tombstone.
MERGE `<dataset>`.`<destinationTable>`
USING (
SELECT batch.key AS key, [partitionTime, ]value
FROM (
SELECT src.i, src.key FROM (
SELECT ARRAY_AGG(
x ORDER BY i DESC LIMIT 1
)[OFFSET(0)] src
FROM (
SELECT * FROM `<dataset>`.`<intermediateTable>`
WHERE batchNumber=<batchNumber>
) x
WHERE x.value IS NULL
GROUP BY key.<field>[, key.<field>...])) AS deletes
RIGHT JOIN (
SELECT * FROM `<dataset>`.`<intermediateTable`
WHERE batchNumber=<batchNumber>
) AS batch
USING (key)
WHERE deletes.i IS NULL OR batch.i >= deletes.i
ORDER BY batch.i ASC) AS src
ON `<destinationTable>`.<keyField>=src.key AND src.value IS NULL
WHEN MATCHED
THEN DELETE
WHEN NOT MATCHED AND src.value IS NOT NULL
THEN INSERT (`<keyField>`, [_PARTITIONTIME, ]`<valueField>`[, `<valueField>`])
VALUES (
src.key,
[CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),]
src.value.<field>[, src.value.<field>...]
);
*/
private String deleteMergeFlushQuery(
TableId intermediateTable, TableId destinationTable, int batchNumber, Schema intermediateSchema
) {
List<String> keyFields = listFields(
intermediateSchema.getFields().get(INTERMEDIATE_TABLE_KEY_FIELD_NAME).getSubFields(),
INTERMEDIATE_TABLE_KEY_FIELD_NAME + "."
);
List<String> valueColumns = valueColumns(intermediateSchema);
final String key = INTERMEDIATE_TABLE_KEY_FIELD_NAME;
final String i = INTERMEDIATE_TABLE_ITERATION_FIELD_NAME;
final String value = INTERMEDIATE_TABLE_VALUE_FIELD_NAME;
final String batch = INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD;
return "MERGE " + table(destinationTable) + " "
+ "USING ("
+ "SELECT batch." + key + " AS " + key + ", " + partitionTimeColumn() + value + " "
+ "FROM ("
+ "SELECT src." + i + ", src." + key + " FROM ("
+ "SELECT ARRAY_AGG("
+ "x ORDER BY " + i + " DESC LIMIT 1"
+ ")[OFFSET(0)] src "
+ "FROM ("
+ "SELECT * FROM " + table(intermediateTable) + " "
+ "WHERE " + batch + "=" + batchNumber
+ ") x "
+ "WHERE x." + value + " IS NULL "
+ "GROUP BY " + String.join(", ", keyFields) + ")) AS deletes "
+ "RIGHT JOIN ("
+ "SELECT * FROM " + table(intermediateTable) + " "
+ "WHERE " + batch + "=" + batchNumber
+ ") AS batch "
+ "USING (" + key + ") "
+ "WHERE deletes." + i + " IS NULL OR batch." + i + " >= deletes." + i + " "
+ "ORDER BY batch." + i + " ASC) AS src "
+ "ON `" + destinationTable.getTable() + "`." + keyFieldName + "=src." + key + " AND src." + value + " IS NULL "
+ "WHEN MATCHED "
+ "THEN DELETE "
+ "WHEN NOT MATCHED AND src." + value + " IS NOT NULL "
+ "THEN INSERT (`"
+ keyFieldName + "`, "
+ partitionTimePseudoColumn()
+ "`"
+ String.join("`, `", valueColumns) + "`) "
+ "VALUES ("
+ "src." + key + ", "
+ partitionTimeValue()
+ valueColumns.stream().map(col -> "src." + value + "." + col).collect(Collectors.joining(", "))
+ ");";
}
private String table(TableId tableId) {
return String.format("`%s`.`%s`", tableId.getDataset(), tableId.getTable());
}
private List<String> valueColumns(Schema intermediateTableSchema) {
return intermediateTableSchema.getFields().get(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).getSubFields()
.stream()
.map(Field::getName)
.collect(Collectors.toList());
}
private String partitionTimePseudoColumn() {
return insertPartitionTime ? "_PARTITIONTIME, " : "";
}
private String partitionTimeValue() {
return insertPartitionTime
? "CAST(CAST(DATE(src." + INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ") AS DATE) AS TIMESTAMP), "
: "";
}
private String partitionTimeColumn() {
return insertPartitionTime
? INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ", "
: "";
}
// DELETE FROM `<dataset>`.`<intermediateTable>` WHERE batchNumber <= <batchNumber> AND _PARTITIONTIME IS NOT NULL;
@VisibleForTesting
static String batchClearQuery(TableId intermediateTable, int batchNumber) {
return new StringBuilder("DELETE FROM `").append(intermediateTable.getDataset()).append("`.`").append(intermediateTable.getTable()).append("` ")
.append("WHERE ")
.append(INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD).append(" <= ").append(batchNumber).append(" ")
// Use this clause to filter out rows that are still in the streaming buffer, which should
// not be subjected to UPDATE or DELETE operations or the query will FAIL
.append("AND _PARTITIONTIME IS NOT NULL")
.append(";")
.toString();
}
private static List<String> listFields(FieldList keyFields, String prefix) {
return keyFields.stream()
.flatMap(field -> {
String fieldName = prefix + field.getName();
FieldList subFields = field.getSubFields();
if (subFields == null) {
return Stream.of(fieldName);
}
return listFields(subFields, fieldName + ".").stream();
}).collect(Collectors.toList());
}
}