From 8ea2f16aa6bc84e162c978bf139d68be71277ab4 Mon Sep 17 00:00:00 2001 From: dhercher Date: Fri, 17 Dec 2021 13:36:35 -0800 Subject: [PATCH 001/145] proper schemaMap handling to allow column renaming in sql template PiperOrigin-RevId: 417051330 --- v2/datastream-to-sql/pom.xml | 17 ++++++++++++ .../v2/templates/DataStreamToSQL.java | 13 ++++++++-- .../v2/utils/DatastreamToDMLTest.java | 26 +++++++++++++++++++ 3 files changed, 54 insertions(+), 2 deletions(-) diff --git a/v2/datastream-to-sql/pom.xml b/v2/datastream-to-sql/pom.xml index 4d40688f6d..ca2935a19f 100644 --- a/v2/datastream-to-sql/pom.xml +++ b/v2/datastream-to-sql/pom.xml @@ -28,6 +28,10 @@ datastream-to-sql + + 1.0.1 + + Stream CDC Data from DataStream to SQL via GCS @@ -49,6 +53,19 @@ mysql-connector-java 8.0.13 + + + com.google.truth + truth + ${truth.version} + test + + + com.google.guava + guava + + + diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java index acf2d8eb22..a0c52930eb 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java @@ -23,6 +23,7 @@ import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.common.base.Splitter; import java.sql.SQLException; +import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -228,6 +229,15 @@ public static void validateOptions( } } + /** Parse the SchemaMap config which allows key:value pairs of column naming configs. */ + public static Map parseSchemaMap(String schemaMapString) { + if (schemaMapString == null || schemaMapString.equals("")) { + return new HashMap<>(); + } + + return Splitter.on(",").withKeyValueSeparator(":").split(schemaMapString); + } + /** * Runs the pipeline with the supplied options. * @@ -247,8 +257,7 @@ public static PipelineResult run(Options options) { CdcJdbcIO.DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(options); validateOptions(options, dataSourceConfiguration); - Map schemaMap = - Splitter.on(",").withKeyValueSeparator(":").split(options.getSchemaMap()); + Map schemaMap = parseSchemaMap(options.getSchemaMap()); /* * Stage 1: Ingest and Normalize Data to FailsafeElement with JSON Strings diff --git a/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java b/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java index 8e935e2cbf..a9a213c731 100644 --- a/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java +++ b/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java @@ -15,10 +15,14 @@ */ package com.google.cloud.teleport.v2.utils; +import static com.google.common.truth.Truth.assertThat; import static org.junit.Assert.assertEquals; +import com.google.cloud.teleport.v2.templates.DataStreamToSQL; import com.google.cloud.teleport.v2.values.DatastreamRow; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.map.ObjectMapper; import org.junit.Test; @@ -104,4 +108,26 @@ public void testGetPostgresTableName() { String tableName = datastreamToDML.getTargetTableName(row); assertEquals(expectedTableName, tableName); } + + /** Test cleaning schema map. */ + @Test + public void testParseSchemaMap() { + Map singleItemExpected = + new HashMap() { + { + put("a", "b"); + } + }; + Map doubleItemExpected = + new HashMap() { + { + put("a", "b"); + put("c", "d"); + } + }; + + assertThat(DataStreamToSQL.parseSchemaMap("")).isEmpty(); + assertThat(DataStreamToSQL.parseSchemaMap("a:b")).isEqualTo(singleItemExpected); + assertThat(DataStreamToSQL.parseSchemaMap("a:b,c:d")).isEqualTo(doubleItemExpected); + } } From ea2a59bd23893df108669e975f74aefe85542a6a Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Mon, 20 Dec 2021 12:45:27 -0800 Subject: [PATCH 002/145] Add parameter which allows users to specify strategy to employ when target file already exists. PiperOrigin-RevId: 417460614 --- v2/common/pom.xml | 6 + .../v2/templates/DataplexBigQueryToGcs.java | 91 +++++++++++++- .../BigQueryTableToGcsTransform.java | 44 ++++++- .../v2/utils/BigQueryToGcsFileNaming.java | 50 ++++++++ .../DataplexBigQueryToGcsFilterTest.java | 114 ++++++++++++++++-- .../templates/DataplexBigQueryToGcsTest.java | 29 ++++- .../v2/utils/BigQueryToGcsFileNamingTest.java | 51 ++++++++ 7 files changed, 364 insertions(+), 21 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNaming.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNamingTest.java diff --git a/v2/common/pom.xml b/v2/common/pom.xml index 389200120a..1fdc9bb1a3 100644 --- a/v2/common/pom.xml +++ b/v2/common/pom.xml @@ -101,6 +101,12 @@ com.google.cloud.bigtable bigtable-hbase-beam ${bigtable.version} + + + org.slf4j + slf4j-log4j12 + + diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index db38f61349..9db248a81b 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -17,6 +17,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static java.util.stream.Collectors.toList; import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1Asset; import com.google.cloud.bigquery.BigQuery; @@ -27,6 +28,7 @@ import com.google.cloud.teleport.v2.clients.DefaultDataplexClient; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; +import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn.BigQueryClientFactory; import com.google.cloud.teleport.v2.transforms.UpdateDataplexBigQueryToGcsExportMetadataTransform; @@ -48,6 +50,10 @@ import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -172,6 +178,14 @@ public interface DataplexBigQueryToGcsOptions String getPartitionIdRegExp(); void setPartitionIdRegExp(String partitionIdRegExp); + + @Description( + "Specifies the action that occurs if destination file already exists. Format: OVERWRITE," + + " FAIL, SKIP. Default: SKIP.") + @Default.Enum("SKIP") + WriteDisposition getWriteDisposition(); + + void setWriteDisposition(WriteDisposition writeDisposition); } /** @@ -242,13 +256,15 @@ private static Pipeline buildPipeline( DataplexAssetResourceSpec.BIGQUERY_DATASET); String targetRootPath = "gs://" + gcsResource; + List existingTargetFiles = getFilesInDirectory(targetRootPath); DatasetId datasetId = BigQueryUtils.parseDatasetUrn(bqResource); BigQueryMetadataLoader metadataLoader = new BigQueryMetadataLoader(bqClient, bqsClient, maxParallelBigQueryRequests); LOG.info("Loading BigQuery metadata..."); List tables = - metadataLoader.loadDatasetMetadata(datasetId, new MetadataFilter(options)); + metadataLoader.loadDatasetMetadata( + datasetId, new MetadataFilter(options, targetRootPath, existingTargetFiles)); LOG.info("Loaded {} table(s).", tables.size()); if (!tables.isEmpty()) { @@ -335,6 +351,24 @@ private static String resolveAsset( return resourceName; } + @VisibleForTesting + static List getFilesInDirectory(String path) { + try { + String pattern = String.format("%s/**", path); + MatchResult result = FileSystems.match(pattern, EmptyMatchTreatment.ALLOW); + List fileNames = + result.metadata().stream() + .map(MatchResult.Metadata::resourceId) + .map(ResourceId::toString) + .collect(toList()); + LOG.info("{} file(s) found in directory {}", fileNames.size(), path); + return fileNames; + } catch (Exception e) { + LOG.error("Exception thrown while getting output files in gcs resource."); + throw new RuntimeException(e); + } + } + @VisibleForTesting static class MetadataFilter implements BigQueryMetadataLoader.Filter { private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); @@ -342,9 +376,16 @@ static class MetadataFilter implements BigQueryMetadataLoader.Filter { private final Instant maxLastModifiedTime; private final Set includeTables; private final Pattern includePartitions; + private final String targetRootPath; + private final String writeDisposition; + private final String fileSuffix; + private final List existingTargetFiles; @VisibleForTesting - MetadataFilter(DataplexBigQueryToGcsOptions options) { + MetadataFilter( + DataplexBigQueryToGcsOptions options, + String targetRootPath, + List existingTargetFiles) { String dateTime = options.getExportDataModifiedBeforeDateTime(); if (dateTime != null && !dateTime.isEmpty()) { if (dateTime.startsWith("-P") || dateTime.startsWith("-p")) { @@ -376,6 +417,10 @@ static class MetadataFilter implements BigQueryMetadataLoader.Filter { } else { this.includePartitions = null; } + this.targetRootPath = targetRootPath; + this.writeDisposition = options.getWriteDisposition().getWriteDisposition(); + this.fileSuffix = options.getFileFormat().getFileSuffix(); + this.existingTargetFiles = existingTargetFiles; } private boolean shouldSkipTableName(BigQueryTable.Builder table) { @@ -385,6 +430,26 @@ private boolean shouldSkipTableName(BigQueryTable.Builder table) { return false; } + private boolean shouldSkipFile(String table, String partition) { + String identifier = partition == null ? table : table + "$" + partition; + switch (writeDisposition) { + case "FAIL": + throw new RuntimeException( + String.format( + "Target File exists for %s. Failing according to writeDisposition", identifier)); + case "SKIP": + LOG.info("Target File exists for %s. Skipping according to writeDisposition", identifier); + return true; + case "OVERWRITE": + LOG.info( + "Target File exists for %s. Overwriting according to writeDisposition", identifier); + return false; + default: + throw new UnsupportedOperationException( + writeDisposition + " writeDisposition not implemented"); + } + } + @Override public boolean shouldSkipUnpartitionedTable(BigQueryTable.Builder table) { if (shouldSkipTableName(table)) { @@ -397,6 +462,14 @@ public boolean shouldSkipUnpartitionedTable(BigQueryTable.Builder table) { && maxLastModifiedTime.isBefore(table.getLastModificationTime() / 1000)) { return true; } + // Check if the target file already exists + String expectedTargetPath = + String.format( + "%s/%s/output-%s%s", + targetRootPath, table.getTableName(), table.getTableName(), fileSuffix); + if (existingTargetFiles.contains(expectedTargetPath)) { + return shouldSkipFile(table.getTableName(), null); + } return false; } @@ -431,6 +504,20 @@ public boolean shouldSkipPartition( includePartitions.pattern()); return true; } + // Check if target file already exists + String expectedTargetPath = + String.format( + "%s/%s/%s_pid=%s/output-%s-%s%s", + targetRootPath, + table.getTableName(), + table.getPartitioningColumn(), + partition.getPartitionName(), + table.getTableName(), + partition.getPartitionName(), + fileSuffix); + if (existingTargetFiles.contains(expectedTargetPath)) { + return shouldSkipFile(table.getTableName(), partition.getPartitionName()); + } return false; } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java index 2177000a5e..9e483b8f57 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.transforms; +import com.google.cloud.teleport.v2.utils.BigQueryToGcsFileNaming; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexCompression; @@ -112,7 +113,13 @@ private PCollection> transformTable( return begin .apply(tableNodeName("Read"), getDefaultRead().from(table.toTableReference())) - .apply(tableNodeName("Write"), getDefaultWrite().via(sink).to(targetPath)) + .apply( + tableNodeName("Write"), + getDefaultWrite() + .via(sink) + .withNaming( + new BigQueryToGcsFileNaming(outputFileFormat.fileSuffix, table.getTableName())) + .to(targetPath)) .getPerDestinationOutputFilenames() .apply( tableNodeName("MapFileNames"), @@ -144,7 +151,16 @@ private PCollection> transformPartition( return begin .apply(partitionNodeName("Read", partition), getDefaultRead().fromQuery(sql)) - .apply(partitionNodeName("Write", partition), getDefaultWrite().via(sink).to(targetPath)) + .apply( + partitionNodeName("Write", partition), + getDefaultWrite() + .via(sink) + .withNaming( + new BigQueryToGcsFileNaming( + outputFileFormat.fileSuffix, + table.getTableName(), + partition.getPartitionName())) + .to(targetPath)) .getPerDestinationOutputFilenames() .apply( partitionNodeName("MapFileNames", partition), @@ -172,8 +188,7 @@ private TypedRead getDefaultRead() { private Write getDefaultWrite() { return FileIO.write() - .withNumShards(1) // Must be 1 as we can only have 1 file per partition. - .withSuffix(outputFileFormat.fileSuffix); + .withNumShards(1); // Must be 1 as we can only have 1 file per partition. } private String tableNodeName(String prefix) { @@ -201,5 +216,26 @@ public enum FileFormat { FileFormat(String fileSuffix) { this.fileSuffix = fileSuffix; } + + public String getFileSuffix() { + return fileSuffix; + } + } + + /** Possible write disposition supported by {@link BigQueryTableToGcsTransform}. */ + public enum WriteDisposition { + OVERWRITE("OVERWRITE"), + SKIP("SKIP"), + FAIL("FAIL"); + + private final String writeDisposition; + + WriteDisposition(String writeDisposition) { + this.writeDisposition = writeDisposition; + } + + public String getWriteDisposition() { + return writeDisposition; + } } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNaming.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNaming.java new file mode 100644 index 0000000000..684898f888 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNaming.java @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2021 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.v2.utils; + +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.Compression; +import org.apache.beam.sdk.io.FileIO.Write.FileNaming; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; + +/** A FileNaming that generates UUID file names for a given directory and file suffix. */ +public class BigQueryToGcsFileNaming implements FileNaming { + + private final String tableName; + private final String partitionName; + private final String suffix; + + public BigQueryToGcsFileNaming(String suffix, String tableName, @Nullable String partitionName) { + this.suffix = suffix; + this.tableName = tableName; + this.partitionName = partitionName; + } + + public BigQueryToGcsFileNaming(String suffix, String tableName) { + this(suffix, tableName, null); + } + + @Override + public String getFilename( + BoundedWindow window, PaneInfo pane, int numShards, int shardIndex, Compression compression) { + String filename = + partitionName != null + ? String.format("output-%s-%s%s", tableName, partitionName, suffix) + : String.format("output-%s%s", tableName, suffix); + return filename; + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java index 37aeb42531..f978502694 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java @@ -19,10 +19,13 @@ import com.google.cloud.teleport.v2.templates.DataplexBigQueryToGcs.DataplexBigQueryToGcsOptions; import com.google.cloud.teleport.v2.templates.DataplexBigQueryToGcs.MetadataFilter; +import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; +import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader.Filter; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -54,7 +57,7 @@ public void test_whenNoFilterOptions_filterAcceptsAllTablesAndPartitions() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(t)).isFalse(); assertThat(f.shouldSkipPartitionedTable(t, Collections.singletonList(p))).isFalse(); @@ -71,7 +74,7 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { options.setTableRefs("includedTable1,includedTable2"); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(includedTable1)).isFalse(); assertThat(f.shouldSkipUnpartitionedTable(includedTable2)).isFalse(); @@ -90,7 +93,7 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { @Test(expected = IllegalArgumentException.class) public void test_whenTableRefsIsInvalid_throwsException() { options.setTableRefs(","); - new MetadataFilter(options); + new MetadataFilter(options, "", new ArrayList()); } @Test @@ -114,7 +117,7 @@ public void test_whenBeforeDateSet_filterExcludesTablesAndPartitions() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); @@ -138,7 +141,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -146,7 +149,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { // Should be the same as 15:00 UTC: options.setExportDataModifiedBeforeDateTime("2021-01-01T14:00:00-01:00"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -154,7 +157,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { // Should be the same as 15:00 UTC: options.setExportDataModifiedBeforeDateTime("2021-01-01T17:00:00+02:00"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -163,7 +166,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { // 14:00 UTC is 1 hour is earlier that both table's last modified time // (14:59:59.999 and 15:00:00.001 UTC). Expecting both to be skipped. options.setExportDataModifiedBeforeDateTime("2021-01-01T14:00:00Z"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); } @@ -180,7 +183,7 @@ public void test_whenBeforeDateHasNoTime_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("2021-02-15"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -196,7 +199,7 @@ public void test_whenBeforeDateIs1DayDuration_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("-P1D"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); } @@ -212,7 +215,7 @@ public void test_whenBeforeDateIs1Day3HoursDuration_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("-p1dt3h"); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); } @@ -222,11 +225,98 @@ public void test_whenPartitionedTableHasNoPartitions_filterExcludesTable() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new MetadataFilter(options); + Filter f = new MetadataFilter(options, "", new ArrayList()); assertThat(f.shouldSkipPartitionedTable(table(), Collections.emptyList())).isTrue(); } + @Test + public void test_whenTargetFileExistsWithWriteDisposionSKIP_filterExcludesTables() { + BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); + BigQueryTablePartition p = partition().setPartitionName("partition1").build(); + + options.setTableRefs(null); + options.setExportDataModifiedBeforeDateTime(null); + options.setFileFormat(FileFormat.AVRO); + options.setWriteDisposition(WriteDisposition.SKIP); + String targetRootPath = "gs://root/"; + String targetfilePathUnpartitionedTable = + String.format("%s/table1/output-table1.avro", targetRootPath); + String targetfilePathPartitionedTable = + String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); + Filter f = + new MetadataFilter( + options, + targetRootPath, + new ArrayList() { + { + add(targetfilePathUnpartitionedTable); + add(targetfilePathPartitionedTable); + } + }); + + assertThat(f.shouldSkipUnpartitionedTable(t)).isTrue(); + assertThat(f.shouldSkipPartition(t, p)).isTrue(); + } + + @Test + public void test_whenTargetFileExistsWithWriteDisposionOverwrite_filterAcceptsTables() { + BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); + BigQueryTablePartition p = partition().setPartitionName("partition1").build(); + + options.setTableRefs(null); + options.setExportDataModifiedBeforeDateTime(null); + options.setFileFormat(FileFormat.AVRO); + options.setWriteDisposition(WriteDisposition.OVERWRITE); + String targetRootPath = "gs://root/"; + String targetfilePathUnpartitionedTable = + String.format("%s/table1/output-table1.avro", targetRootPath); + String targetfilePathPartitionedTable = + String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); + Filter f = + new MetadataFilter( + options, + targetRootPath, + new ArrayList() { + { + add(targetfilePathUnpartitionedTable); + add(targetfilePathPartitionedTable); + } + }); + + assertThat(f.shouldSkipUnpartitionedTable(t)).isFalse(); + assertThat(f.shouldSkipPartition(t, p)).isFalse(); + } + + @Test(expected = RuntimeException.class) + public void test_whenTargetFileExistsWithWriteDisposionFail_filterAcceptsTables() { + BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); + BigQueryTablePartition p = partition().setPartitionName("partition1").build(); + + options.setTableRefs(null); + options.setExportDataModifiedBeforeDateTime(null); + options.setFileFormat(FileFormat.AVRO); + options.setWriteDisposition(WriteDisposition.FAIL); + String targetRootPath = "gs://root/"; + String targetfilePathUnpartitionedTable = + String.format("%s/table1/output-table1.avro", targetRootPath); + String targetfilePathPartitionedTable = + String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); + Filter f = + new MetadataFilter( + options, + targetRootPath, + new ArrayList() { + { + add(targetfilePathUnpartitionedTable); + add(targetfilePathPartitionedTable); + } + }); + + f.shouldSkipUnpartitionedTable(t); + f.shouldSkipPartition(t, p); + } + private static BigQueryTable.Builder table() { return BigQueryTable.builder().setTableName("defaultTableName").setLastModificationTime(0L); } diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java index 1e66a0d6dc..39da9a2945 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.common.truth.Truth.assertThat; import static org.mockito.Mockito.any; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -235,7 +236,9 @@ public void testE2E_mainPathWithAllStepsEnabled() throws Exception { .apply( "readTableFiles", ParquetIO.read(avroSchema) - .from(outDir.getAbsolutePath() + "/unpartitioned_table/*.parquet")) + .from( + outDir.getAbsolutePath() + + "/unpartitioned_table/output-unpartitioned_table.parquet")) .apply( "mapTableFiles", MapElements.into(TypeDescriptors.strings()).via(Object::toString)); PCollection actualPartitionedRecords1 = @@ -243,14 +246,18 @@ public void testE2E_mainPathWithAllStepsEnabled() throws Exception { .apply( "readP1Files", ParquetIO.read(avroSchema) - .from(outDir.getAbsolutePath() + "/partitioned_table/ts_pid=p1/*.parquet")) + .from( + outDir.getAbsolutePath() + + "/partitioned_table/ts_pid=p1/output-partitioned_table-p1.parquet")) .apply("mapP1Files", MapElements.into(TypeDescriptors.strings()).via(Object::toString)); PCollection actualPartitionedRecords2 = testPipeline .apply( "readP2Files", ParquetIO.read(avroSchema) - .from(outDir.getAbsolutePath() + "/partitioned_table/ts_pid=p2/*.parquet")) + .from( + outDir.getAbsolutePath() + + "/partitioned_table/ts_pid=p2/output-partitioned_table-p2.parquet")) .apply("mapP2Files", MapElements.into(TypeDescriptors.strings()).via(Object::toString)); PCollection actualPartitionedRecords3 = testPipeline @@ -316,6 +323,22 @@ public void testE2E_withDeleteSourceDataDisabled_doesntTruncateData() throws Exc verifyNoMoreInteractions(bqMock); } + @Test + public void testGetFilesInDirectory_withValidPath_returnsPathsOfFilesInDirectory() + throws Exception { + File outputDir1 = tmpDir.newFolder("out", "unpartitioned_table"); + File outputFile1 = + new File(outputDir1.getAbsolutePath() + "/" + "output-unpartitioned_table.parquet"); + outputFile1.createNewFile(); + File outputDir2 = tmpDir.newFolder("out", "partitioned_table", "p2_pid=partition"); + File outputFile2 = + new File(outputDir2.getAbsolutePath() + "/" + "output-partitioned_table-partition.parquet"); + outputFile2.createNewFile(); + + List files = DataplexBigQueryToGcs.getFilesInDirectory(outDir.getAbsolutePath()); + assertThat(files.size()).isEqualTo(2); + } + private void insertTableData(String tableName, TableRow... records) throws Exception { fakeDatasetService.insertAll( tableByName.get(tableName).toTableReference(), Arrays.asList(records), null); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNamingTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNamingTest.java new file mode 100644 index 0000000000..881dc2b84b --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsFileNamingTest.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2021 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.v2.utils; + +import static org.junit.Assert.assertEquals; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link BigQueryToGcsFileNaming}. */ +@RunWith(JUnit4.class) +public class BigQueryToGcsFileNamingTest { + + @Before + public void setUp() {} + + @Test + public void testGetFilenameWithoutPartition() { + String tableName = "people"; + String suffix = ".avro"; + String expectedFilename = "output-people.avro"; + BigQueryToGcsFileNaming fileNaming = new BigQueryToGcsFileNaming(suffix, tableName); + assertEquals(fileNaming.getFilename(null, null, 0, 0, null), expectedFilename); + } + + @Test + public void testGetFilenameWithPartition() { + String tableName = "people"; + String partitionName = "name_partition"; + String suffix = ".avro"; + String expectedFilename = "output-people-name_partition.avro"; + BigQueryToGcsFileNaming fileNaming = + new BigQueryToGcsFileNaming(suffix, tableName, partitionName); + assertEquals(fileNaming.getFilename(null, null, 0, 0, null), expectedFilename); + } +} From 68e6939d82e1ad84aa69f33e59c74d6bdea89a3e Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 20 Dec 2021 13:05:15 -0800 Subject: [PATCH 003/145] Adding tempDir parameter in Spanner to avro export template. PiperOrigin-RevId: 417464200 --- .../teleport/spanner/ExportPipeline.java | 8 ++++++- .../teleport/spanner/ExportTransform.java | 23 +++++++++++++------ .../teleport/util/ValueProviderUtils.java | 21 +++++++++++++++++ .../spanner/ExportRelatedTablesCheckTest.java | 3 ++- .../teleport/spanner/ExportTimestampTest.java | 4 +++- 5 files changed, 49 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java index c739658cca..1300fe1ff3 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java @@ -45,6 +45,11 @@ public interface ExportPipelineOptions extends PipelineOptions { void setOutputDir(ValueProvider value); + @Description("Temporary Directory to store Avro files.") + ValueProvider getAvroTempDirectory(); + + void setAvroTempDirectory(ValueProvider value); + @Description("Test dataflow job identifier for Beam Direct Runner") @Default.String(value = "") ValueProvider getTestJobId(); @@ -132,7 +137,8 @@ public static void main(String[] args) { options.getSnapshotTime(), options.getTableNames(), options.getShouldExportRelatedTables(), - options.getShouldExportTimestampAsLogicalType())); + options.getShouldExportTimestampAsLogicalType(), + options.getAvroTempDirectory())); PipelineResult result = p.run(); if (options.getWaitUntilFinish() && diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java index 8ee6081e9d..0f90f78172 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner; import static com.google.cloud.teleport.spanner.SpannerTableFilter.getFilteredTables; +import static com.google.cloud.teleport.util.ValueProviderUtils.eitherOrValueProvider; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.TimestampBound; @@ -127,6 +128,7 @@ public class ExportTransform extends PTransform private final ValueProvider tableNames; private final ValueProvider exportRelatedTables; private final ValueProvider shouldExportTimestampAsLogicalType; + private final ValueProvider avroTempDirectory; public ExportTransform( SpannerConfig spannerConfig, @@ -139,7 +141,8 @@ public ExportTransform( /*snapshotTime=*/ ValueProvider.StaticValueProvider.of(""), /*tableNames=*/ ValueProvider.StaticValueProvider.of(""), /*exportRelatedTables=*/ ValueProvider.StaticValueProvider.of(false), - /*shouldExportTimestampAsLogicalType=*/ ValueProvider.StaticValueProvider.of(false)); + /*shouldExportTimestampAsLogicalType=*/ ValueProvider.StaticValueProvider.of(false), + outputDir); } public ExportTransform( @@ -149,7 +152,8 @@ public ExportTransform( ValueProvider snapshotTime, ValueProvider tableNames, ValueProvider exportRelatedTables, - ValueProvider shouldExportTimestampAsLogicalType) { + ValueProvider shouldExportTimestampAsLogicalType, + ValueProvider avroTempDirectory) { this.spannerConfig = spannerConfig; this.outputDir = outputDir; this.testJobId = testJobId; @@ -157,6 +161,7 @@ public ExportTransform( this.tableNames = tableNames; this.exportRelatedTables = exportRelatedTables; this.shouldExportTimestampAsLogicalType = shouldExportTimestampAsLogicalType; + this.avroTempDirectory = avroTempDirectory; } /** @@ -347,12 +352,17 @@ public void processElement(ProcessContext c) { outputDir, (SerializableFunction) s -> FileSystems.matchNewResource(s, true)); + ValueProvider tempResource = + ValueProvider.NestedValueProvider.of( + eitherOrValueProvider(avroTempDirectory, outputDir), + (SerializableFunction) s -> FileSystems.matchNewResource(s, true)); + WriteFilesResult fileWriteResults = rows.apply( "Store Avro files", AvroIO.writeCustomTypeToGenericRecords() .to(new SchemaBasedDynamicDestinations(avroSchemas, outputDirectoryName, resource)) - .withTempDirectory(resource)); + .withTempDirectory(tempResource)); // Generate the manifest file. PCollection>> tableFiles = @@ -494,7 +504,7 @@ private OutputStream createOutputStream(Path outputPath, ProcessContext c) Contextful.of( tableManifestNaming, Requirements.requiresSideInputs(outputDirectoryName))) .via(Contextful.fn(KV::getValue), TextIO.sink()) - .withTempDirectory(outputDir)); + .withTempDirectory(eitherOrValueProvider(avroTempDirectory, outputDir))); PCollection> metadataTables = tableManifests.apply( @@ -519,10 +529,9 @@ private OutputStream createOutputStream(Path outputPath, ProcessContext c) .by(SerializableFunctions.constant("")) .withDestinationCoder(StringUtf8Coder.of()) .via(TextIO.sink()) - .withTempDirectory(outputDir) .withNaming( - Contextful.of( - manifestNaming, Requirements.requiresSideInputs(outputDirectoryName)))); + Contextful.of(manifestNaming, Requirements.requiresSideInputs(outputDirectoryName))) + .withTempDirectory(eitherOrValueProvider(avroTempDirectory, outputDir))); return fileWriteResults; } diff --git a/src/main/java/com/google/cloud/teleport/util/ValueProviderUtils.java b/src/main/java/com/google/cloud/teleport/util/ValueProviderUtils.java index abfbb48ee9..5ac1a6a34f 100644 --- a/src/main/java/com/google/cloud/teleport/util/ValueProviderUtils.java +++ b/src/main/java/com/google/cloud/teleport/util/ValueProviderUtils.java @@ -54,4 +54,25 @@ public String apply(TranslatorInput input) { } }); } + + /** + * Gets a {@link ValueProvider} for an optional parameter. If optionalParameter is available, it + * is returned as is, otherwise defaultParameter is returned instead. + * + * @param optionalParameter + * @param defaultParameter + * @return {@link ValueProvider} + */ + public static ValueProvider eitherOrValueProvider( + ValueProvider optionalParameter, ValueProvider defaultParameter) { + return DualInputNestedValueProvider.of( + optionalParameter, + defaultParameter, + new SerializableFunction, T>() { + @Override + public T apply(TranslatorInput input) { + return (input.getX() != null) ? input.getX() : input.getY(); + } + }); + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java index f3301acab8..d8e0f05149 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java @@ -1141,7 +1141,8 @@ private void exportAndImportDb( timestamp, tables, exportRelatedTables, - exportAsLogicalType)); + exportAsLogicalType, + destination)); PipelineResult exportResult = exportPipeline.run(); exportResult.waitUntilFinish(); diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java index 84e1322f64..bd7c842334 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java @@ -278,6 +278,7 @@ private void exportAndImportDbAtTime( ValueProvider.StaticValueProvider exportAsLogicalType = ValueProvider.StaticValueProvider.of(false); SpannerConfig sourceConfig = spannerServer.getSpannerConfig(sourceDb); + ValueProvider.StaticValueProvider tempDir = ValueProvider.StaticValueProvider.of(null); exportPipeline.apply( "Export", new ExportTransform( @@ -287,7 +288,8 @@ private void exportAndImportDbAtTime( timestamp, tables, exportRelatedTables, - exportAsLogicalType)); + exportAsLogicalType, + tempDir)); PipelineResult exportResult = exportPipeline.run(); exportResult.waitUntilFinish(); From 1c1cc26e92a99288b9c70c948d0369a7d9db9a06 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Wed, 22 Dec 2021 12:40:54 -0800 Subject: [PATCH 004/145] Exclude and ban vulnerable log4j dependencies. PiperOrigin-RevId: 417869426 --- pom.xml | 1 + v2/hive-to-bigquery/pom.xml | 15 +++++++++++++++ v2/pom.xml | 1 + 3 files changed, 17 insertions(+) diff --git a/pom.xml b/pom.xml index b7c025502f..d1df694d5c 100644 --- a/pom.xml +++ b/pom.xml @@ -797,6 +797,7 @@ com.google.guava:guava-jdk5 com.google.protobuf:protobuf-lite + org.apache.logging.log4j:*:[2.0,2.14.1] diff --git a/v2/hive-to-bigquery/pom.xml b/v2/hive-to-bigquery/pom.xml index ad1d4dd98a..48a6d6e509 100644 --- a/v2/hive-to-bigquery/pom.xml +++ b/v2/hive-to-bigquery/pom.xml @@ -37,6 +37,21 @@ 1.0-SNAPSHOT compile + + org.apache.logging.log4j + log4j-1.2-api + 2.17.0 + + + org.apache.logging.log4j + log4j-web + 2.17.0 + + + org.apache.logging.log4j + log4j-slf4j-impl + 2.17.0 + org.apache.hive hive-common diff --git a/v2/pom.xml b/v2/pom.xml index 08054bcf03..177022a5d7 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -520,6 +520,7 @@ com.google.guava:guava-jdk5 com.google.protobuf:protobuf-lite + org.apache.logging.log4j:*:[2.0,2.14.1] From 9dd14da865f47f6ba33f0280c509b89a0482d20d Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Wed, 22 Dec 2021 23:05:10 -0800 Subject: [PATCH 005/145] Adding tempDir parameter in Spanner to csv export template. PiperOrigin-RevId: 417947120 --- .../teleport/templates/SpannerToText.java | 28 ++++++++++++++++--- 1 file changed, 24 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java index b6e6e9fb9a..367ef0f066 100644 --- a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java +++ b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java @@ -15,6 +15,8 @@ */ package com.google.cloud.teleport.templates; +import static com.google.cloud.teleport.util.ValueProviderUtils.eitherOrValueProvider; + import com.google.cloud.teleport.templates.common.SpannerConverters; import com.google.cloud.teleport.templates.common.SpannerConverters.CreateTransactionFnWithTimestamp; import com.google.cloud.teleport.templates.common.SpannerConverters.SpannerReadOptions; @@ -22,16 +24,20 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.spanner.LocalSpannerIO; import org.apache.beam.sdk.io.gcp.spanner.ReadOperation; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.Transaction; +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.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -70,9 +76,14 @@ public class SpannerToText { /** Custom PipelineOptions. */ public interface SpannerToTextOptions - extends PipelineOptions, - SpannerReadOptions, - FilesystemWriteOptions {} + extends PipelineOptions, SpannerReadOptions, FilesystemWriteOptions { + + @Description("Temporary Directory to store Csv files.") + ValueProvider getCsvTempDirectory(); + + @SuppressWarnings("unused") + void setCsvTempDirectory(ValueProvider value); + } /** * Runs a pipeline which reads in Records from Spanner, and writes the CSV to TextIO sink. @@ -135,8 +146,17 @@ public static void main(String[] args) { MapElements.into(TypeDescriptors.strings()) .via(struct -> (new SpannerConverters.StructCsvPrinter()).print(struct))); + ValueProvider tempDirectoryResource = + ValueProvider.NestedValueProvider.of( + eitherOrValueProvider(options.getCsvTempDirectory(), options.getTextWritePrefix()), + (SerializableFunction) s -> FileSystems.matchNewResource(s, true)); + csv.apply( - "Write to storage", TextIO.write().to(options.getTextWritePrefix()).withSuffix(".csv")); + "Write to storage", + TextIO.write() + .to(options.getTextWritePrefix()) + .withSuffix(".csv") + .withTempDirectory(tempDirectoryResource)); pipeline.run(); LOG.info("Completed pipeline setup"); From 7fe8f4f4de3ce26d63c2cf274e74f4372cd06a96 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Thu, 23 Dec 2021 23:41:08 -0800 Subject: [PATCH 006/145] Support for passing priority parameter in Import/Export spanner templates. PiperOrigin-RevId: 418121236 --- .../teleport/spanner/ExportPipeline.java | 9 ++++- .../teleport/spanner/ImportPipeline.java | 17 ++++++--- .../teleport/spanner/TextImportPipeline.java | 9 ++++- .../teleport/templates/SpannerToText.java | 9 ++++- .../io/gcp/spanner/LocalBatchSpannerRead.java | 36 +++++++++++++++++++ .../sdk/io/gcp/spanner/LocalSpannerIO.java | 11 +++++- .../sdk/io/gcp/spanner/SpannerConfig.java | 16 +++++++++ 7 files changed, 98 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java index 1300fe1ff3..260e426bd4 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner; +import com.google.cloud.spanner.Options.RpcPriority; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -107,6 +108,11 @@ public interface ExportPipelineOptions extends PipelineOptions { ValueProvider getShouldExportRelatedTables(); void setShouldExportRelatedTables(ValueProvider value); + + @Description("The spanner priority. --spannerPriority must be one of:[HIGH,MEDIUM,LOW]") + ValueProvider getSpannerPriority(); + + void setSpannerPriority(ValueProvider value); } /** @@ -126,7 +132,8 @@ public static void main(String[] args) { .withProjectId(options.getSpannerProjectId()) .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) - .withDatabaseId(options.getDatabaseId()); + .withDatabaseId(options.getDatabaseId()) + .withRpcPriority(options.getSpannerPriority()); p.begin() .apply( "Run Export", diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java index 7b30cb8d8e..b13d072802 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner; +import com.google.cloud.spanner.Options.RpcPriority; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -63,10 +64,10 @@ public interface Options extends PipelineOptions { void setWaitForIndexes(ValueProvider value); @Description( - "By default the import pipeline is not blocked on foreign key creation, and it " - + "may complete with foreign keys still being created in the background. In testing, it may " - + "be useful to set this option to false so that the pipeline waits until foreign keys are " - + "finished.") + "By default the import pipeline is not blocked on foreign key creation, and it may complete" + + " with foreign keys still being created in the background. In testing, it may be" + + " useful to set this option to false so that the pipeline waits until foreign keys" + + " are finished.") @Default.Boolean(false) ValueProvider getWaitForForeignKeys(); @@ -97,6 +98,11 @@ public interface Options extends PipelineOptions { ValueProvider getDDLCreationTimeoutInMinutes(); void setDDLCreationTimeoutInMinutes(ValueProvider value); + + @Description("The spanner priority. --spannerPriority must be one of:[HIGH,MEDIUM,LOW]") + ValueProvider getSpannerPriority(); + + void setSpannerPriority(ValueProvider value); } public static void main(String[] args) { @@ -110,7 +116,8 @@ public static void main(String[] args) { .withProjectId(options.getSpannerProjectId()) .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) - .withDatabaseId(options.getDatabaseId()); + .withDatabaseId(options.getDatabaseId()) + .withRpcPriority(options.getSpannerPriority()); p.apply( new ImportTransform( diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java index 82d32e0a9c..cf63b0ce4c 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner; +import com.google.cloud.spanner.Options.RpcPriority; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -163,6 +164,11 @@ public interface Options extends PipelineOptions { boolean getWaitUntilFinish(); void setWaitUntilFinish(boolean value); + + @Description("The spanner priority. --spannerPriority must be one of:[HIGH,MEDIUM,LOW]") + ValueProvider getSpannerPriority(); + + void setSpannerPriority(ValueProvider value); } public static void main(String[] args) { @@ -175,7 +181,8 @@ public static void main(String[] args) { SpannerConfig.create() .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) - .withDatabaseId(options.getDatabaseId()); + .withDatabaseId(options.getDatabaseId()) + .withRpcPriority(options.getSpannerPriority()); p.apply(new TextImportTransform(spannerConfig, options.getImportManifest())); diff --git a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java index 367ef0f066..8ab8cda275 100644 --- a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java +++ b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java @@ -17,6 +17,7 @@ import static com.google.cloud.teleport.util.ValueProviderUtils.eitherOrValueProvider; +import com.google.cloud.spanner.Options.RpcPriority; import com.google.cloud.teleport.templates.common.SpannerConverters; import com.google.cloud.teleport.templates.common.SpannerConverters.CreateTransactionFnWithTimestamp; import com.google.cloud.teleport.templates.common.SpannerConverters.SpannerReadOptions; @@ -83,6 +84,11 @@ public interface SpannerToTextOptions @SuppressWarnings("unused") void setCsvTempDirectory(ValueProvider value); + + @Description("The spanner priority. --spannerPriority must be one of:[HIGH,MEDIUM,LOW]") + ValueProvider getSpannerPriority(); + + void setSpannerPriority(ValueProvider value); } /** @@ -104,7 +110,8 @@ public static void main(String[] args) { .withHost(options.getSpannerHost()) .withProjectId(options.getSpannerProjectId()) .withInstanceId(options.getSpannerInstanceId()) - .withDatabaseId(options.getSpannerDatabaseId()); + .withDatabaseId(options.getSpannerDatabaseId()) + .withRpcPriority(options.getSpannerPriority()); PTransform> spannerExport = SpannerConverters.ExportTransformFactory.create( diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java index d0e0dc60dc..c67b78a1c3 100644 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java +++ b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java @@ -17,6 +17,8 @@ import com.google.auto.value.AutoValue; import com.google.cloud.spanner.BatchReadOnlyTransaction; +import com.google.cloud.spanner.Options; +import com.google.cloud.spanner.Options.RpcPriority; import com.google.cloud.spanner.Partition; import com.google.cloud.spanner.ResultSet; import com.google.cloud.spanner.Struct; @@ -110,6 +112,14 @@ public void processElement(ProcessContext c) throws Exception { } private List execute(ReadOperation op, BatchReadOnlyTransaction tx) { + if (config.getRpcPriority() != null && config.getRpcPriority().get() != null) { + return executeWithPriority(op, tx, config.getRpcPriority().get()); + } else { + return executeWithoutPriority(op, tx); + } + } + + private List executeWithoutPriority(ReadOperation op, BatchReadOnlyTransaction tx) { // Query was selected. if (op.getQuery() != null) { return tx.partitionQuery(op.getPartitionOptions(), op.getQuery()); @@ -127,6 +137,32 @@ private List execute(ReadOperation op, BatchReadOnlyTransaction tx) { return tx.partitionRead( op.getPartitionOptions(), op.getTable(), op.getKeySet(), op.getColumns()); } + + private List executeWithPriority( + ReadOperation op, BatchReadOnlyTransaction tx, RpcPriority rpcPriority) { + // Query was selected. + if (op.getQuery() != null) { + return tx.partitionQuery( + op.getPartitionOptions(), op.getQuery(), Options.priority(rpcPriority)); + } + // Read with index was selected. + if (op.getIndex() != null) { + return tx.partitionReadUsingIndex( + op.getPartitionOptions(), + op.getTable(), + op.getIndex(), + op.getKeySet(), + op.getColumns(), + Options.priority(rpcPriority)); + } + // Read from table was selected. + return tx.partitionRead( + op.getPartitionOptions(), + op.getTable(), + op.getKeySet(), + op.getColumns(), + Options.priority(rpcPriority)); + } } private static class ReadFromPartitionFn extends DoFn { diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java index d775e4f35f..b550027c15 100644 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java +++ b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java @@ -29,6 +29,7 @@ import com.google.cloud.spanner.KeySet; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Mutation.Op; +import com.google.cloud.spanner.Options; import com.google.cloud.spanner.PartitionOptions; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerException; @@ -1483,7 +1484,15 @@ private void spannerWriteWithRetryIfSchemaChange(Iterable batch) throws SpannerException { for (int retry = 1; ; retry++) { try { - spannerAccessor.getDatabaseClient().writeAtLeastOnce(batch); + if (spannerConfig.getRpcPriority() != null + && spannerConfig.getRpcPriority().get() != null) { + spannerAccessor + .getDatabaseClient() + .writeAtLeastOnceWithOptions( + batch, Options.priority(spannerConfig.getRpcPriority().get())); + } else { + spannerAccessor.getDatabaseClient().writeAtLeastOnce(batch); + } return; } catch (AbortedException e) { if (retry >= ABORTED_RETRY_ATTEMPTS) { diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index 5a2279e1b4..e1c1a85810 100644 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -19,6 +19,7 @@ import com.google.auto.value.AutoValue; import com.google.cloud.ServiceFactory; +import com.google.cloud.spanner.Options.RpcPriority; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import java.io.Serializable; @@ -40,6 +41,8 @@ public abstract class SpannerConfig implements Serializable { private static final Duration DEFAULT_COMMIT_DEADLINE = Duration.standardSeconds(15); // Total allowable backoff time. private static final Duration DEFAULT_MAX_CUMULATIVE_BACKOFF = Duration.standardMinutes(15); + // A default priority for batch traffic. + private static final RpcPriority DEFAULT_RPC_PRIORITY = RpcPriority.MEDIUM; public abstract @Nullable ValueProvider getProjectId(); @@ -55,6 +58,8 @@ public abstract class SpannerConfig implements Serializable { public abstract @Nullable ValueProvider getMaxCumulativeBackoff(); + public abstract @Nullable ValueProvider getRpcPriority(); + @VisibleForTesting abstract @Nullable ServiceFactory getServiceFactory(); @@ -66,6 +71,7 @@ public static SpannerConfig create() { .setCommitDeadline(ValueProvider.StaticValueProvider.of(DEFAULT_COMMIT_DEADLINE)) .setMaxCumulativeBackoff( ValueProvider.StaticValueProvider.of(DEFAULT_MAX_CUMULATIVE_BACKOFF)) + .setRpcPriority(ValueProvider.StaticValueProvider.of(DEFAULT_RPC_PRIORITY)) .build(); } @@ -115,6 +121,8 @@ public abstract static class Builder { abstract Builder setServiceFactory(ServiceFactory serviceFactory); + abstract Builder setRpcPriority(ValueProvider rpcPriority); + public abstract SpannerConfig build(); } @@ -170,4 +178,12 @@ public SpannerConfig withMaxCumulativeBackoff(ValueProvider maxCumulat SpannerConfig withServiceFactory(ServiceFactory serviceFactory) { return toBuilder().setServiceFactory(serviceFactory).build(); } + + public SpannerConfig withRpcPriority(ValueProvider rpcPriority) { + return toBuilder().setRpcPriority(rpcPriority).build(); + } + + public SpannerConfig withRpcPriority(RpcPriority rpcPriority) { + return withRpcPriority(ValueProvider.StaticValueProvider.of(rpcPriority)); + } } From 5c5fef92dbef6e25f5363700cc6910fdc5b73bb5 Mon Sep 17 00:00:00 2001 From: dhercher Date: Sat, 25 Dec 2021 14:04:58 -0800 Subject: [PATCH 007/145] empty string is translated to null for psql decimals PiperOrigin-RevId: 418301939 --- .../FormatDatastreamJsonToJson.java | 1 + .../FormatDatastreamJsonToJsonTest.java | 73 +++++++++++++++++++ .../teleport/v2/utils/DatastreamToDML.java | 25 +++++-- .../v2/utils/DatastreamToPostgresDML.java | 18 +++++ .../v2/utils/DatastreamToDMLTest.java | 10 ++- 5 files changed, 116 insertions(+), 11 deletions(-) create mode 100644 v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java index b8f920fd20..1df507bca2 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java @@ -189,6 +189,7 @@ private String getSourceType(JsonNode record) { private long convertTimestampStringToSeconds(String timestamp) { ZonedDateTime zonedDateTime; try { + timestamp = timestamp.replace(" ", "T"); if (!timestamp.endsWith("Z")) { timestamp = timestamp + "Z"; } diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java new file mode 100644 index 0000000000..2eb9807688 --- /dev/null +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2020 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.v2.transforms; + +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for FormatDatastreamRecordToJson function. These check appropriate Avro-to-Json conv. */ +@RunWith(JUnit4.class) +public class FormatDatastreamJsonToJsonTest { + + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static final String EXAMPLE_DATASTREAM_JSON = + "{\"uuid\":\"00c32134-f50e-4460-a6c0-399900010010\",\"read_timestamp\":\"2021-12-25" + + " 05:42:04.408\"," + + "\"source_timestamp\":\"2021-12-25T05:42:04.408\",\"object\":\"HR_JOBS\",\"read_method\":\"oracle-backfill\",\"stream_name\":\"projects/402074789819/locations/us-central1/streams/destroy\",\"schema_key\":\"ebdb5545a7610cee7b1caae4a45dec7fd3b46fdc\",\"sort_keys\":[1640410924408,1706664,\"\",0],\"source_metadata\":{\"schema\":\"HR\",\"table\":\"JOBS\",\"database\":\"XE\",\"row_id\":\"AAAEARAAEAAAAC9AAS\",\"scn\":1706664,\"is_deleted\":false,\"change_type\":\"INSERT\",\"ssn\":0,\"rs_id\":\"\",\"tx_id\":null,\"log_file\":\"\",\"primary_keys\":[\"JOB_ID\"]},\"payload\":{\"JOB_ID\":\"PR_REP\",\"JOB_TITLE\":\"Public" + + " Relations Representative\",\"MIN_SALARY\":4500,\"MAX_SALARY\":10500}}"; + + private static final String EXAMPLE_DATASTREAM_RECORD = + "{\"_metadata_stream\":\"my-stream\",\"_metadata_timestamp\":1640410924,\"_metadata_read_timestamp\":1640410924,\"_metadata_read_method\":\"oracle-backfill\",\"_metadata_source_type\":\"oracle\",\"_metadata_deleted\":false,\"_metadata_table\":\"JOBS\",\"_metadata_change_type\":\"INSERT\",\"_metadata_primary_keys\":[\"JOB_ID\"],\"_metadata_schema\":\"HR\",\"_metadata_row_id\":\"AAAEARAAEAAAAC9AAS\",\"_metadata_scn\":1706664,\"_metadata_ssn\":0,\"_metadata_rs_id\":\"\",\"_metadata_tx_id\":null,\"JOB_ID\":\"PR_REP\",\"JOB_TITLE\":\"Public" + + " Relations" + + " Representative\",\"MIN_SALARY\":4500,\"MAX_SALARY\":10500,\"_metadata_source\":{\"schema\":\"HR\",\"table\":\"JOBS\",\"database\":\"XE\",\"row_id\":\"AAAEARAAEAAAAC9AAS\",\"scn\":1706664,\"is_deleted\":false,\"change_type\":\"INSERT\",\"ssn\":0,\"rs_id\":\"\",\"tx_id\":null,\"log_file\":\"\",\"primary_keys\":[\"JOB_ID\"]}}"; + + @Test + public void testProcessElement_validJson() { + Map hashedColumns = new HashMap(); + + FailsafeElement expectedElement = + FailsafeElement.of(EXAMPLE_DATASTREAM_RECORD, EXAMPLE_DATASTREAM_RECORD); + + PCollection> pCollection = + pipeline + .apply("CreateInput", Create.of(EXAMPLE_DATASTREAM_JSON)) + .apply( + "FormatDatastreamJsonToJson", + ParDo.of( + FormatDatastreamJsonToJson.create() + .withStreamName("my-stream") + .withHashColumnValues(hashedColumns) + .withLowercaseSourceColumns(false))) + .setCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); + + PAssert.that(pCollection).containsInAnyOrder(expectedElement); + + pipeline.run(); + } +} diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java index 6c24434744..ab14282c32 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java @@ -214,8 +214,8 @@ public DmlInfo convertJsonToDmlInfo(JsonNode rowObj, String failsafeValue) { List primaryKeys = this.getPrimaryKeys(catalogName, schemaName, tableName, rowObj); List orderByFields = row.getSortFields(); - List primaryKeyValues = getFieldValues(rowObj, primaryKeys); - List orderByValues = getFieldValues(rowObj, orderByFields); + List primaryKeyValues = getFieldValues(rowObj, primaryKeys, tableSchema); + List orderByValues = getFieldValues(rowObj, orderByFields, tableSchema); String dmlSqlTemplate = getDmlTemplate(rowObj, primaryKeys); Map sqlTemplateValues = @@ -278,8 +278,7 @@ public Map getSqlTemplateValues( return sqlTemplateValues; } - public static String getValueSql( - JsonNode rowObj, String columnName, Map tableSchema) { + public String getValueSql(JsonNode rowObj, String columnName, Map tableSchema) { String columnValue; JsonNode columnObj = rowObj.get(columnName); @@ -287,16 +286,24 @@ public static String getValueSql( LOG.warn("Missing Required Value: {} in {}", columnName, rowObj.toString()); return ""; } - if (columnObj.isTextual()) { columnValue = "\'" + cleanSql(columnObj.getTextValue()) + "\'"; } else { columnValue = columnObj.toString(); } + return cleanDataTypeValueSql(columnValue, columnName, tableSchema); + } + + public String cleanDataTypeValueSql( + String columnValue, String columnName, Map tableSchema) { return columnValue; } + public String getNullValueSql() { + return "NULL"; + } + public static String cleanSql(String str) { if (str == null) { return null; @@ -310,11 +317,12 @@ public static String escapeSql(String str) { return StringUtils.replace(str, "'", "''"); } - public List getFieldValues(JsonNode rowObj, List fieldNames) { + public List getFieldValues( + JsonNode rowObj, List fieldNames, Map tableSchema) { List fieldValues = new ArrayList(); for (String fieldName : fieldNames) { - fieldValues.add(getValueSql(rowObj, fieldName, null)); + fieldValues.add(getValueSql(rowObj, fieldName, tableSchema)); } return fieldValues; @@ -538,7 +546,8 @@ private List getTablePrimaryKeys( if (retriesRemaining > 0) { int sleepSecs = (MAX_RETRIES - retriesRemaining + 1) * 10; LOG.info( - "SQLException, will retry after {} seconds: Failed to Retrieve Primary Key: {}.{} : {}", + "SQLException, will retry after {} seconds: Failed to Retrieve Primary Key: {}.{} :" + + " {}", sleepSecs, schemaName, tableName, diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java index 4e9f68a77a..f8cd88f56e 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java @@ -17,6 +17,7 @@ import com.google.cloud.teleport.v2.io.CdcJdbcIO.DataSourceConfiguration; import com.google.cloud.teleport.v2.values.DatastreamRow; +import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,4 +73,21 @@ public String getTargetTableName(DatastreamRow row) { String tableName = row.getTableName(); return cleanTableName(tableName); } + + @Override + public String cleanDataTypeValueSql( + String columnValue, String columnName, Map tableSchema) { + String dataType = tableSchema.get(columnName); + if (dataType == null) { + return columnValue; + } + switch (dataType.toUpperCase()) { + case "DECIMAL": + if (columnValue.equals("")) { + return getNullValueSql(); + } + break; + } + return columnValue; + } } diff --git a/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java b/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java index a9a213c731..edd0c2ed82 100644 --- a/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java +++ b/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java @@ -63,19 +63,23 @@ public void testGetValueSql() { JsonNode rowObj = this.getRowObj(); String expectedTextContent = "'value'"; - String testSqlContent = DatastreamToPostgresDML.getValueSql(rowObj, "text_column", null); + String testSqlContent = + DatastreamToPostgresDML.of(null) + .getValueSql(rowObj, "text_column", new HashMap()); assertEquals(expectedTextContent, testSqlContent); // Single quotes are escaped by 2 single quotes in SQL String expectedQuotedTextContent = "'Test Values: ''!@#$%^'"; String testQuotedSqlContent = - DatastreamToPostgresDML.getValueSql(rowObj, "quoted_text_column", null); + DatastreamToPostgresDML.of(null) + .getValueSql(rowObj, "quoted_text_column", new HashMap()); assertEquals(expectedQuotedTextContent, testQuotedSqlContent); // Null bytes are escaped with blanks values String expectedNullByteTextContent = "'Test Values: Hes made'"; String testNullByteSqlContent = - DatastreamToPostgresDML.getValueSql(rowObj, "null_byte_text_column", null); + DatastreamToPostgresDML.of(null) + .getValueSql(rowObj, "null_byte_text_column", new HashMap()); assertEquals(expectedNullByteTextContent, testNullByteSqlContent); } From 30c493dae07a8768e109460fc4589f4800e1cad6 Mon Sep 17 00:00:00 2001 From: olegsa Date: Wed, 29 Dec 2021 03:16:05 -0800 Subject: [PATCH 008/145] Dataplex File Conversion template supports SKIP/OVERWRITE/FAIL The CL updates the Dataplex File Conversion template to let the user specify how the job should handle the existing output files for the given input files. The options are: SKIP -- skip conversion of the existing files, OVERWRITE -- overwrite the existing files and FAIL -- the job fails if any existing files are detected. PiperOrigin-RevId: 418759903 --- .../DataplexFileFormatConversion.java | 161 ++++++++++++++---- .../DataplexFileFormatConversionTest.java | 124 +++++++++++++- .../entity2.existing/1.json | 1 + .../entity2.existing/1.parquet | Bin 0 -> 656 bytes 4 files changed, 247 insertions(+), 39 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.json create mode 100644 v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.parquet diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java index 7bf4bc38ea..ed35d4464a 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java @@ -32,10 +32,14 @@ import com.google.common.base.Splitter; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Set; +import java.util.function.Predicate; import java.util.regex.Pattern; +import java.util.stream.Stream; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.Pipeline; @@ -45,6 +49,10 @@ import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.FileIO.Sink; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.parquet.ParquetIO; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -97,6 +105,14 @@ public interface FileFormatConversionOptions String getOutputAsset(); void setOutputAsset(String outputAsset); + + @Description( + "Specifies the behaviour if output files already exist. Format: OVERWRITE," + + " FAIL, SKIP. Default: OVERWRITE.") + @Default.Enum("SKIP") + ExistingOutputFilesBehaviour getWriteDisposition(); + + void setWriteDisposition(ExistingOutputFilesBehaviour value); } /** Supported input file formats. */ @@ -120,6 +136,16 @@ public enum OutputFileFormat { } } + /** The enum that defines how to handle existing output files. */ + public enum ExistingOutputFilesBehaviour { + OVERWRITE, + SKIP, + FAIL + } + + private static final ImmutableSet EXPECTED_INPUT_FILES_EXTENSIONS = + ImmutableSet.of(".csv", ".json", ".parquet", ".avro"); + private static final Pattern ASSET_PATTERN = Pattern.compile( "^projects/[^\\n\\r/]+/locations/[^\\n\\r/]+/lakes/[^\\n\\r/]+/zones/[^\\n\\r/]+" @@ -191,6 +217,49 @@ public static PipelineResult run( } String outputBucket = outputAsset.getResourceSpec().getName(); + Predicate inputFilesFilter; + switch (options.getWriteDisposition()) { + case OVERWRITE: + inputFilesFilter = inputFilePath -> true; + break; + case FAIL: + Set outputFilePaths = + getFilesFromFilePattern(addWildCard(outputBucket)) + .collect(ImmutableSet.toImmutableSet()); + inputFilesFilter = + inputFilePath -> { + if (outputFilePaths.contains( + inputFilePathToOutputFilePath( + outputPathProvider, + inputFilePath, + outputBucket, + options.getOutputFileFormat()))) { + throw new RuntimeException( + String.format( + "The file %s already exists in the output asset bucket: %s", + inputFilePath, outputBucket)); + } + return true; + }; + break; + case SKIP: + outputFilePaths = + getFilesFromFilePattern(addWildCard(outputBucket)) + .collect(ImmutableSet.toImmutableSet()); + inputFilesFilter = + inputFilePath -> + !outputFilePaths.contains( + inputFilePathToOutputFilePath( + outputPathProvider, + inputFilePath, + outputBucket, + options.getOutputFileFormat())); + break; + default: + throw new IllegalArgumentException( + "Unsupported existing file behaviour: " + options.getWriteDisposition()); + } + ImmutableList entities = isInputAsset ? dataplex.getCloudStorageEntities(options.getInputAssetOrEntitiesList()) @@ -201,24 +270,25 @@ public static PipelineResult run( ImmutableList partitions = dataplex.getPartitions(entity.getName()); if (partitions.isEmpty()) { - pipeline.apply( - "Convert " + shortenDataplexName(entity.getName()), - new ConvertFiles( - entity, - entityToFileSpec(entity), - options.getOutputFileFormat(), - options.getOutputFileCompression(), - outputPathProvider.outputPathFrom(entity.getDataPath(), outputBucket))); + String outputPath = outputPathProvider.outputPathFrom(entity.getDataPath(), outputBucket); + getFilesFromFilePattern(entityToFileSpec(entity)) + .filter(inputFilesFilter) + .forEach( + inputFilePath -> + pipeline.apply( + "Convert " + shortenDataplexName(entity.getName()), + new ConvertFiles(entity, inputFilePath, options, outputPath))); } else { for (GoogleCloudDataplexV1Partition partition : partitions) { - pipeline.apply( - "Convert " + shortenDataplexName(partition.getName()), - new ConvertFiles( - entity, - partitionToFileSpec(partition), - options.getOutputFileFormat(), - options.getOutputFileCompression(), - outputPathProvider.outputPathFrom(partition.getLocation(), outputBucket))); + String outputPath = + outputPathProvider.outputPathFrom(partition.getLocation(), outputBucket); + getFilesFromFilePattern(partitionToFileSpec(partition)) + .filter(inputFilesFilter) + .forEach( + inputFilePath -> + pipeline.apply( + "Convert " + shortenDataplexName(partition.getName()), + new ConvertFiles(entity, inputFilePath, options, outputPath))); } } } @@ -255,6 +325,33 @@ private static String ensurePathEndsWithSlash(String path) { return path.endsWith("/") ? path : path + '/'; } + /** Example conversion: 1.json => 1.parquet; 1.abc => 1.abc.parquet. */ + private static String replaceInputExtensionWithOutputExtension( + String path, OutputFileFormat outputFileFormat) { + String inputFileExtension = path.substring(path.lastIndexOf('.')); + if (EXPECTED_INPUT_FILES_EXTENSIONS.contains(inputFileExtension)) { + return path.substring(0, path.length() - inputFileExtension.length()) + + outputFileFormat.fileSuffix; + } else { + return path + outputFileFormat.fileSuffix; + } + } + + private static String inputFilePathToOutputFilePath( + OutputPathProvider outputPathProvider, + String inputFilePath, + String outputBucket, + OutputFileFormat outputFileFormat) { + return replaceInputExtensionWithOutputExtension( + outputPathProvider.outputPathFrom(inputFilePath, outputBucket), outputFileFormat); + } + + private static Stream getFilesFromFilePattern(String pattern) throws IOException { + return FileSystems.match(pattern, EmptyMatchTreatment.ALLOW).metadata().stream() + .map(MatchResult.Metadata::resourceId) + .map(ResourceId::toString); + } + /** Convert the input file path to a new output file path. */ @FunctionalInterface interface OutputPathProvider { @@ -269,22 +366,21 @@ private static class ConvertFiles extends PTransform { private static final TupleTag CSV_LINES = new TupleTag() {}; private final GoogleCloudDataplexV1Entity entity; - private final String inputFileSpec; + private final String inputFilePath; private final OutputFileFormat outputFileFormat; private final DataplexCompression outputFileCompression; private final String outputPath; protected ConvertFiles( GoogleCloudDataplexV1Entity entity, - String inputFileSpec, - OutputFileFormat outputFileFormat, - DataplexCompression outputFileCompression, + String inputFilePath, + FileFormatConversionOptions options, String outputPath) { super(); this.entity = entity; - this.outputFileFormat = outputFileFormat; - this.inputFileSpec = inputFileSpec; - this.outputFileCompression = outputFileCompression; + this.outputFileFormat = options.getOutputFileFormat(); + this.inputFilePath = inputFilePath; + this.outputFileCompression = options.getOutputFileCompression(); this.outputPath = outputPath; } @@ -297,7 +393,7 @@ public PDone expand(PBegin input) { case CSV: records = input - .apply("CSV", readCsvTransform(entity, inputFileSpec)) + .apply("CSV", readCsvTransform(entity, inputFilePath)) .get(CSV_LINES) .apply("ToGenRec", ParDo.of(csvToGenericRecordFn(entity, serializedSchema))) .setCoder(AvroCoder.of(GenericRecord.class, schema)); @@ -305,7 +401,7 @@ public PDone expand(PBegin input) { case JSON: records = input - .apply("Json", readJsonTransform(inputFileSpec)) + .apply("Json", readJsonTransform(inputFilePath)) .apply("ToGenRec", ParDo.of(jsonToGenericRecordFn(serializedSchema))) .setCoder(AvroCoder.of(GenericRecord.class, schema)); break; @@ -314,7 +410,7 @@ public PDone expand(PBegin input) { input.apply( "Parquet", ParquetConverters.ReadParquetFile.newBuilder() - .withInputFileSpec(inputFileSpec) + .withInputFileSpec(inputFilePath) .withSerializedSchema(serializedSchema) .build()); break; @@ -323,7 +419,7 @@ public PDone expand(PBegin input) { input.apply( "Avro", AvroConverters.ReadAvroFile.newBuilder() - .withInputFileSpec(inputFileSpec) + .withInputFileSpec(inputFilePath) .withSerializedSchema(serializedSchema) .build()); break; @@ -348,13 +444,16 @@ public PDone expand(PBegin input) { "Output format is not implemented: " + outputFileFormat); } + String outputFileName = + replaceInputExtensionWithOutputExtension( + inputFilePath.substring(inputFilePath.lastIndexOf('/') + 1), outputFileFormat); + records.apply( "Write", FileIO.write() .via(sink) .to(ensurePathEndsWithSlash(outputPath)) - .withPrefix(entityToOutputFilePrefix(entity)) - .withSuffix(outputFileFormat.fileSuffix) + .withNaming((window, pane, numShards, shardIndex, compression) -> outputFileName) .withNumShards(1)); // Must be 1 as we can only have 1 file per Dataplex partition. return PDone.in(input.getPipeline()); @@ -410,9 +509,5 @@ private static JsonConverters.StringToGenericRecordFn jsonToGenericRecordFn( String serializedSchema) { return new JsonConverters.StringToGenericRecordFn(serializedSchema); } - - private static String entityToOutputFilePrefix(GoogleCloudDataplexV1Entity entity) { - return entity.getName().substring(entity.getName().lastIndexOf('/') + 1); - } } } diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java index 7ff726ebbf..feaa2a4fb9 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java @@ -26,6 +26,7 @@ import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1SchemaSchemaField; import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1StorageFormat; import com.google.cloud.teleport.v2.clients.DataplexClient; +import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.ExistingOutputFilesBehaviour; import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.FileFormatConversionOptions; import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.InputFileFormat; import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.OutputFileFormat; @@ -38,6 +39,10 @@ import com.google.common.io.Resources; import java.io.File; import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import org.apache.avro.Schema; import org.apache.avro.Schema.Parser; import org.apache.avro.generic.GenericData.Record; @@ -86,19 +91,20 @@ public class DataplexFileFormatConversionTest { private static final GoogleCloudDataplexV1Entity entity1 = new GoogleCloudDataplexV1Entity() - .setName("projects/p1/locations/l1/lakes/l1}/zones/z1/entities/e1") + .setName("projects/p1/locations/l1/lakes/l1/zones/z1/entities/e1") + .setId("e1") .setSystem(StorageSystem.CLOUD_STORAGE.name()) .setFormat(new GoogleCloudDataplexV1StorageFormat().setFormat(InputFileFormat.CSV.name())) .setSchema(SCHEMA); private static final GoogleCloudDataplexV1Partition partition11 = new GoogleCloudDataplexV1Partition() - .setName("projects/p1/locations/l1/lakes/l1}/zones/z1/entities/e1/partitions/p11") + .setName("projects/p1/locations/l1/lakes/l1/zones/z1/entities/e1/partitions/p11") .setLocation(Resources.getResource(RESOURCES_DIR + "/entity1/partition11").getPath()); private static final GoogleCloudDataplexV1Partition partition12 = new GoogleCloudDataplexV1Partition() - .setName("projects/p1/locations/l1/lakes/l1}/zones/z1/entities/e1/partitions/p12") + .setName("projects/p1/locations/l1/lakes/l1/zones/z1/entities/e1/partitions/p12") .setLocation(Resources.getResource(RESOURCES_DIR + "/entity1/partition12").getPath()); private static final GoogleCloudDataplexV1Asset asset2 = @@ -106,7 +112,8 @@ public class DataplexFileFormatConversionTest { .setName("projects/p1/locations/l1/lakes/l1/zones/z1/assets/a2"); private static final GoogleCloudDataplexV1Entity entity2 = new GoogleCloudDataplexV1Entity() - .setName("projects/p1/locations/l1/lakes/l1}/zones/z1/entities/e2") + .setName("projects/p1/locations/l1/lakes/l1/zones/z1/entities/e2") + .setId("e2") .setAsset(asset2.getName()) .setSystem(StorageSystem.CLOUD_STORAGE.name()) .setFormat( @@ -282,6 +289,103 @@ public void testAssetWithEntityParquetToAvroE2E() throws IOException { readPipeline.run(); } + /** + * Tests JSON to Parquet conversion for an asset with entity when one of the files already exists + * and the existing file behaviour is SKIP. + */ + @Test + @Category(NeedsRunner.class) + public void testAssetWithEntityJsonToParquetSkipExistingFilesE2E() throws IOException { + // setup Dataplex client to return entity 2 + DataplexClient dataplex = mock(DataplexClient.class); + when(dataplex.getCloudStorageEntities(asset2.getName())).thenReturn(ImmutableList.of(entity2)); + when(dataplex.getPartitions(entity2.getName())).thenReturn(ImmutableList.of()); + when(dataplex.getAsset(outputAsset.getName())).thenReturn(outputAsset); + + // setup options to skip existing files + FileFormatConversionOptions options = + PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); + options.setInputAssetOrEntitiesList(asset2.getName()); + options.setOutputFileFormat(OutputFileFormat.PARQUET); + options.setOutputAsset(outputAsset.getName()); + options.setWriteDisposition(ExistingOutputFilesBehaviour.SKIP); + + // simulate the file 1.json -> 1.parquet already + copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/1.parquet"); + + // run the pipeline, only 2.json -> 2.parquet conversion should happen + DataplexFileFormatConversion.run( + mainPipeline, options, dataplex, DataplexFileFormatConversionTest::outputPathProvider); + + // read the conversion results + PCollection readParquetFile = + readPipeline.apply( + "ReadParquetFile", + ParquetConverters.ReadParquetFile.newBuilder() + .withInputFileSpec(temporaryFolder.getRoot().getAbsolutePath() + "/**/*.parquet") + .withSerializedSchema(EXPECT_SERIALIZED_AVRO_SCHEMA) + .build()); + + // expect old 1.parquet (from entity2.existing) and newly converted 2.parquet (from entity2) + ImmutableList.Builder expected = ImmutableList.builder(); + Record record = new Record(EXPECTED_AVRO_SCHEMA); + record.put("Word", "abc.existing"); + record.put("Number", 1); + expected.add(record); + record = new Record(EXPECTED_AVRO_SCHEMA); + record.put("Word", "def"); + record.put("Number", 2); + expected.add(record); + record = new Record(EXPECTED_AVRO_SCHEMA); + record.put("Word", "ghi"); + record.put("Number", 3); + expected.add(record); + + PAssert.that(readParquetFile).containsInAnyOrder(expected.build()); + + readPipeline.run(); + } + + /** + * Tests JSON to Parquet conversion for an asset with entity when one of the files already exists + * and the existing file behaviour is FAIL. + */ + @Test(expected = RuntimeException.class) + @Category(NeedsRunner.class) + public void testAssetWithEntityJsonToParquetFailOnExistingFilesE2E() throws IOException { + // setup Dataplex client to return entity 2 + DataplexClient dataplex = mock(DataplexClient.class); + when(dataplex.getCloudStorageEntities(asset2.getName())).thenReturn(ImmutableList.of(entity2)); + when(dataplex.getPartitions(entity2.getName())).thenReturn(ImmutableList.of()); + when(dataplex.getAsset(outputAsset.getName())).thenReturn(outputAsset); + + // setup options to skip existing files + FileFormatConversionOptions options = + PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); + options.setInputAssetOrEntitiesList(asset2.getName()); + options.setOutputFileFormat(OutputFileFormat.PARQUET); + options.setOutputAsset(outputAsset.getName()); + options.setWriteDisposition(ExistingOutputFilesBehaviour.FAIL); + + // simulate the file 1.json -> 1.parquet already + copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/1.parquet"); + + // run the pipeline, only 2.json -> 2.parquet conversion should happen + DataplexFileFormatConversion.run( + mainPipeline, options, dataplex, DataplexFileFormatConversionTest::outputPathProvider) + .waitUntilFinish(); + } + + private void copyFileToOutputBucket(String sourceRelativePath, String destinationRelativePath) + throws IOException { + Path source = + Paths.get(Resources.getResource(RESOURCES_DIR + '/' + sourceRelativePath).getPath()); + Path destination = + Paths.get(temporaryFolder.getRoot().getAbsolutePath() + '/' + destinationRelativePath); + Files.createDirectories(destination.getParent()); + Files.copy(source, destination); + } + private static ImmutableList expectedGenericRecords() { ImmutableList.Builder expected = ImmutableList.builder(); Record record = new Record(EXPECTED_AVRO_SCHEMA); @@ -300,8 +404,16 @@ record = new Record(EXPECTED_AVRO_SCHEMA); } private static String outputPathProvider(String inputPath, String outputBucket) { - String relativeInputPath = - new File(RESOURCES_DIR).toURI().relativize(new File(inputPath).toURI()).getPath(); + String relativeInputPath; + try { + relativeInputPath = + Resources.getResource(RESOURCES_DIR) + .toURI() + .relativize(new File(inputPath).toURI()) + .getPath(); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } return outputBucket + '/' + relativeInputPath; } } diff --git a/v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.json b/v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.json new file mode 100644 index 0000000000..d952b51e5d --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.json @@ -0,0 +1 @@ +{"Word": "abc.existing", "Number": 111} \ No newline at end of file diff --git a/v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.parquet b/v2/googlecloud-to-googlecloud/src/test/resources/DataplexFileFormatConversionTest/entity2.existing/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e133a429ccbd493e602154ed6dc895b5e78c873c GIT binary patch literal 656 zcmZ8fO^ee|6uk*?2$IDN&VA%T79rR$LJUo1YFoOQorsj7E`*AUm*z!-O*_7%qcU_M z%yJg9@E3@nf5Kle|G>2y*KWM|P+K>7Iq#fz@44r_)01h0fG*y}4>w=G-EvSTXrc+Q zKehls#Pd+T%Zf5vzN{GwKYjW2^Y)g|urw0#1D)H}w){VcI%loSs1Mln(r!D<#3}K> z9X7*J^+(M3(caVfE13&40Id$##Sld5@@L~_wB}32>p>f| zUF>sJ?7M~`uuF~UzR9k%LrNg9@0)DhUPFTC>one;}Ul5>i|3%sSs zE8Jv6IL+iDDZJ71i$7GA7ZL5jqEvK`-X{3VH}U;bWXsaqyx4$!@K$A|RAIhKl#dyfZnk=0%PpcipUZ}B(g?3Jqk literal 0 HcmV?d00001 From eab2393693d5b60b2506c934cd11866568cab5ef Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Wed, 29 Dec 2021 11:11:48 -0800 Subject: [PATCH 009/145] Add tests to increase coverage in DataplexBigQueryToGcs template. PiperOrigin-RevId: 418820756 --- .../options/DataplexBigQueryToGcsOptions.java | 121 +++++++ .../v2/templates/DataplexBigQueryToGcs.java | 301 ++---------------- .../v2/utils/DataplexBigQueryToGcsFilter.java | 185 +++++++++++ .../templates/DataplexBigQueryToGcsTest.java | 79 ++++- .../DataplexBigQueryToGcsFilterTest.java | 37 ++- 5 files changed, 428 insertions(+), 295 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java rename v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/{templates => utils}/DataplexBigQueryToGcsFilterTest.java (90%) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java new file mode 100644 index 0000000000..e11ccb31a7 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2021 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.v2.options; + +import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; +import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; +import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn; +import com.google.cloud.teleport.v2.transforms.UpdateDataplexBigQueryToGcsExportMetadataTransform; +import com.google.cloud.teleport.v2.values.DataplexCompression; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.ExperimentalOptions; +import org.apache.beam.sdk.options.Validation.Required; + +/** + * The {@link DataplexBigQueryToGcsOptions} class provides the custom execution options passed by + * the executor at the command-line. + */ +public interface DataplexBigQueryToGcsOptions + extends GcpOptions, + ExperimentalOptions, + DeleteBigQueryDataFn.Options, + UpdateDataplexBigQueryToGcsExportMetadataTransform.Options { + + @Description( + "Dataplex asset name for the the BigQuery dataset to tier data from. Format:" + + " projects//locations//lakes//zones//assets/.") + @Required + String getSourceBigQueryAssetName(); + + void setSourceBigQueryAssetName(String sourceBigQueryAssetName); + + @Description( + "A comma-separated list of BigQuery tables to tier. If none specified, all tables will be" + + " tiered. Tables should be specified by their name only (no project/dataset prefix)." + + " Case-sensitive!") + String getTableRefs(); + + void setTableRefs(String tableRefs); + + @Description( + "Dataplex asset name for the the GCS bucket to tier data to. Format:" + + " projects//locations//lakes//zones//assets/.") + @Required + String getDestinationGcsBucketAssetName(); + + void setDestinationGcsBucketAssetName(String destinationGcsBucketAssetName); + + @Description( + "The parameter can either be: 1) unspecified, 2) date (and optional time) 3) Duration.\n" + + "1) If not specified move all tables / partitions.\n" + + "2) Move data older than this date (and optional time). For partitioned tables, move" + + " partitions last modified before this date/time. For non-partitioned tables, move if" + + " the table was last modified before this date/time. If not specified, move all tables" + + " / partitions. The date/time is parsed in the default time zone by default, but" + + " optinal suffixes Z and +HH:mm are supported. Format: YYYY-MM-DD or" + + " YYYY-MM-DDTHH:mm:ss or YYYY-MM-DDTHH:mm:ss+03:00.\n" + + "3) Similar to the above (2) but the effective date-time is derived from the current" + + " time in the default/system timezone shifted by the provided duration in the format" + + " based on ISO-8601 +/-PnDTnHnMn.nS " + + "(https://docs.oracle.com/javase/8/docs/api/java/time/Duration.html#parse-java.lang.CharSequence-)." + + " However only \"minus\" durations are accepted so only past effective date-times are" + + " possible.") + String getExportDataModifiedBeforeDateTime(); + + void setExportDataModifiedBeforeDateTime(String exportDataModifiedBeforeDateTime); + + @Description( + "The maximum number of parallel requests that will be sent to BigQuery when loading" + + " table/partition metadata. Default: 5.") + @Default.Integer(5) + @Required + Integer getMaxParallelBigQueryMetadataRequests(); + + void setMaxParallelBigQueryMetadataRequests(Integer maxParallelBigQueryMetadataRequests); + + @Description("Output file format in GCS. Format: PARQUET, AVRO, or ORC. Default: PARQUET.") + @Default.Enum("PARQUET") + @Required + FileFormat getFileFormat(); + + void setFileFormat(FileFormat fileFormat); + + @Description( + "Output file compression. Format: UNCOMPRESSED, SNAPPY, GZIP, or BZIP2. Default:" + + " SNAPPY. BZIP2 not supported for PARQUET files.") + @Default.Enum("SNAPPY") + DataplexCompression getFileCompression(); + + void setFileCompression(DataplexCompression fileCompression); + + @Description( + "Process partitions with partition ID matching this regexp only. Default: process all.") + String getPartitionIdRegExp(); + + void setPartitionIdRegExp(String partitionIdRegExp); + + @Description( + "Specifies the action that occurs if destination file already exists. Format: OVERWRITE," + + " FAIL, SKIP. Default: SKIP.") + @Default.Enum("SKIP") + WriteDisposition getWriteDisposition(); + + void setWriteDisposition(WriteDisposition writeDisposition); +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index 9db248a81b..f5320e69c3 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -26,40 +26,29 @@ import com.google.cloud.bigquery.storage.v1beta1.BigQueryStorageClient; import com.google.cloud.teleport.v2.clients.DataplexClient; import com.google.cloud.teleport.v2.clients.DefaultDataplexClient; +import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn.BigQueryClientFactory; import com.google.cloud.teleport.v2.transforms.UpdateDataplexBigQueryToGcsExportMetadataTransform; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader; import com.google.cloud.teleport.v2.utils.BigQueryUtils; +import com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexAssetResourceSpec; -import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Splitter; -import com.google.re2j.Pattern; import java.io.IOException; -import java.time.Duration; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; -import java.util.Set; import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation.Required; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; @@ -71,8 +60,6 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import org.joda.time.Instant; -import org.joda.time.format.ISODateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,100 +81,6 @@ public class DataplexBigQueryToGcs { private static final Logger LOG = LoggerFactory.getLogger(DataplexBigQueryToGcs.class); - /** - * The {@link DataplexBigQueryToGcsOptions} class provides the custom execution options passed by - * the executor at the command-line. - */ - public interface DataplexBigQueryToGcsOptions - extends GcpOptions, - ExperimentalOptions, - DeleteBigQueryDataFn.Options, - UpdateDataplexBigQueryToGcsExportMetadataTransform.Options { - - @Description( - "Dataplex asset name for the the BigQuery dataset to tier data from. Format:" - + " projects//locations//lakes//zones//assets/.") - @Required - String getSourceBigQueryAssetName(); - - void setSourceBigQueryAssetName(String sourceBigQueryAssetName); - - @Description( - "A comma-separated list of BigQuery tables to tier. If none specified, all tables will be" - + " tiered. Tables should be specified by their name only (no project/dataset prefix)." - + " Case-sensitive!") - String getTableRefs(); - - void setTableRefs(String tableRefs); - - @Description( - "Dataplex asset name for the the GCS bucket to tier data to. Format:" - + " projects//locations//lakes//zones//assets/.") - @Required - String getDestinationGcsBucketAssetName(); - - void setDestinationGcsBucketAssetName(String destinationGcsBucketAssetName); - - @Description( - "The parameter can either be: 1) unspecified, 2) date (and optional time) 3) Duration.\n" - + "1) If not specified move all tables / partitions.\n" - + "2) Move data older than this date (and optional time). For partitioned tables, move" - + " partitions last modified before this date/time. For non-partitioned tables, move if" - + " the table was last modified before this date/time. If not specified, move all" - + " tables / partitions. The date/time is parsed in the default time zone by default," - + " but optinal suffixes Z and +HH:mm are supported. Format: YYYY-MM-DD or" - + " YYYY-MM-DDTHH:mm:ss or YYYY-MM-DDTHH:mm:ss+03:00.\n" - + "3) Similar to the above (2) but the effective date-time is derived from the current" - + " time in the default/system timezone shifted by the provided duration in the format" - + " based on ISO-8601 +/-PnDTnHnMn.nS " - + "(https://docs.oracle.com/javase/8/docs/api/java/time/Duration.html#parse-java.lang.CharSequence-)." - + " However only \"minus\" durations are accepted so only past effective date-times are" - + " possible.") - String getExportDataModifiedBeforeDateTime(); - - void setExportDataModifiedBeforeDateTime(String exportDataModifiedBeforeDateTime); - - @Description( - "The maximum number of parallel requests that will be sent to BigQuery when loading" - + " table/partition metadata. Default: 5.") - @Default.Integer(5) - @Required - Integer getMaxParallelBigQueryMetadataRequests(); - - void setMaxParallelBigQueryMetadataRequests(Integer maxParallelBigQueryMetadataRequests); - - @Description("Output file format in GCS. Format: PARQUET, AVRO, or ORC. Default: PARQUET.") - @Default.Enum("PARQUET") - @Required - FileFormat getFileFormat(); - - void setFileFormat(FileFormat fileFormat); - - @Description( - "Output file compression. Format: UNCOMPRESSED, SNAPPY, GZIP, or BZIP2. Default:" - + " SNAPPY. BZIP2 not supported for PARQUET files.") - @Default.Enum("SNAPPY") - DataplexCompression getFileCompression(); - - void setFileCompression(DataplexCompression fileCompression); - - @Description( - "Process partitions with partition ID matching this regexp only. Default: process all.") - String getPartitionIdRegExp(); - - void setPartitionIdRegExp(String partitionIdRegExp); - - @Description( - "Specifies the action that occurs if destination file already exists. Format: OVERWRITE," - + " FAIL, SKIP. Default: SKIP.") - @Default.Enum("SKIP") - WriteDisposition getWriteDisposition(); - - void setWriteDisposition(WriteDisposition writeDisposition); - } - /** * Main entry point for pipeline execution. * @@ -216,28 +109,20 @@ public static void main(String[] args) BigQuery bqClient = BigQueryOptions.getDefaultInstance().getService(); try (BigQueryStorageClient bqsClient = BigQueryStorageClient.create()) { LOG.info("Building the pipeline..."); - pipeline = buildPipeline(options, dataplex, bqClient, bqsClient); + pipeline = setUpPipeline(options, dataplex, bqClient, bqsClient); } LOG.info("Running the pipeline."); pipeline.run(); } - /** - * Builds the pipeline with the supplied options. - * - * @param options The execution parameters to the pipeline. - * @return The resulting pipeline. - */ - private static Pipeline buildPipeline( + private static Pipeline setUpPipeline( DataplexBigQueryToGcsOptions options, DataplexClient dataplex, BigQuery bqClient, BigQueryStorageClient bqsClient) throws IOException, ExecutionException, InterruptedException { - Pipeline pipeline = Pipeline.create(options); - int maxParallelBigQueryRequests = options.getMaxParallelBigQueryMetadataRequests(); checkArgument( maxParallelBigQueryRequests >= 1, @@ -249,6 +134,7 @@ private static Pipeline buildPipeline( dataplex, options.getDestinationGcsBucketAssetName(), DataplexAssetResourceSpec.STORAGE_BUCKET); + String bqResource = resolveAsset( dataplex, @@ -258,13 +144,35 @@ private static Pipeline buildPipeline( String targetRootPath = "gs://" + gcsResource; List existingTargetFiles = getFilesInDirectory(targetRootPath); DatasetId datasetId = BigQueryUtils.parseDatasetUrn(bqResource); + BigQueryMetadataLoader metadataLoader = new BigQueryMetadataLoader(bqClient, bqsClient, maxParallelBigQueryRequests); + return buildPipeline(options, metadataLoader, targetRootPath, datasetId); + } + + /** + * Builds the pipeline with the supplied options. + * + * @param options The execution parameters to the pipeline. + * @return The resulting pipeline. + */ + @VisibleForTesting + static Pipeline buildPipeline( + DataplexBigQueryToGcsOptions options, + BigQueryMetadataLoader metadataLoader, + String targetRootPath, + DatasetId datasetId) + throws ExecutionException, InterruptedException { + + Pipeline pipeline = Pipeline.create(options); + List existingTargetFiles = getFilesInDirectory(targetRootPath); + LOG.info("Loading BigQuery metadata..."); List tables = metadataLoader.loadDatasetMetadata( - datasetId, new MetadataFilter(options, targetRootPath, existingTargetFiles)); + datasetId, + new DataplexBigQueryToGcsFilter(options, targetRootPath, existingTargetFiles)); LOG.info("Loaded {} table(s).", tables.size()); if (!tables.isEmpty()) { @@ -368,157 +276,4 @@ static List getFilesInDirectory(String path) { throw new RuntimeException(e); } } - - @VisibleForTesting - static class MetadataFilter implements BigQueryMetadataLoader.Filter { - private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - - private final Instant maxLastModifiedTime; - private final Set includeTables; - private final Pattern includePartitions; - private final String targetRootPath; - private final String writeDisposition; - private final String fileSuffix; - private final List existingTargetFiles; - - @VisibleForTesting - MetadataFilter( - DataplexBigQueryToGcsOptions options, - String targetRootPath, - List existingTargetFiles) { - String dateTime = options.getExportDataModifiedBeforeDateTime(); - if (dateTime != null && !dateTime.isEmpty()) { - if (dateTime.startsWith("-P") || dateTime.startsWith("-p")) { - this.maxLastModifiedTime = Instant.now().plus(Duration.parse(dateTime).toMillis()); - } else { - this.maxLastModifiedTime = - Instant.parse(dateTime, ISODateTimeFormat.dateOptionalTimeParser()); - } - } else { - this.maxLastModifiedTime = null; - } - - String tableRefs = options.getTableRefs(); - if (tableRefs != null && !tableRefs.isEmpty()) { - List tableRefList = SPLITTER.splitToList(tableRefs); - checkArgument( - !tableRefList.isEmpty(), - "Got an non-empty tableRefs param '%s', but couldn't parse it into a valid table list," - + " please check its format.", - tableRefs); - this.includeTables = new HashSet<>(tableRefList); - } else { - this.includeTables = null; - } - - String partitionRegExp = options.getPartitionIdRegExp(); - if (partitionRegExp != null && !partitionRegExp.isEmpty()) { - this.includePartitions = Pattern.compile(partitionRegExp); - } else { - this.includePartitions = null; - } - this.targetRootPath = targetRootPath; - this.writeDisposition = options.getWriteDisposition().getWriteDisposition(); - this.fileSuffix = options.getFileFormat().getFileSuffix(); - this.existingTargetFiles = existingTargetFiles; - } - - private boolean shouldSkipTableName(BigQueryTable.Builder table) { - if (includeTables != null && !includeTables.contains(table.getTableName())) { - return true; - } - return false; - } - - private boolean shouldSkipFile(String table, String partition) { - String identifier = partition == null ? table : table + "$" + partition; - switch (writeDisposition) { - case "FAIL": - throw new RuntimeException( - String.format( - "Target File exists for %s. Failing according to writeDisposition", identifier)); - case "SKIP": - LOG.info("Target File exists for %s. Skipping according to writeDisposition", identifier); - return true; - case "OVERWRITE": - LOG.info( - "Target File exists for %s. Overwriting according to writeDisposition", identifier); - return false; - default: - throw new UnsupportedOperationException( - writeDisposition + " writeDisposition not implemented"); - } - } - - @Override - public boolean shouldSkipUnpartitionedTable(BigQueryTable.Builder table) { - if (shouldSkipTableName(table)) { - return true; - } - // Check the last modified time only for NOT partitioned table. - // If a table is partitioned, we check the last modified time on partition level only. - if (maxLastModifiedTime != null - // BigQuery timestamps are in microseconds so / 1000. - && maxLastModifiedTime.isBefore(table.getLastModificationTime() / 1000)) { - return true; - } - // Check if the target file already exists - String expectedTargetPath = - String.format( - "%s/%s/output-%s%s", - targetRootPath, table.getTableName(), table.getTableName(), fileSuffix); - if (existingTargetFiles.contains(expectedTargetPath)) { - return shouldSkipFile(table.getTableName(), null); - } - return false; - } - - @Override - public boolean shouldSkipPartitionedTable( - BigQueryTable.Builder table, List partitions) { - if (shouldSkipTableName(table)) { - return true; - } - if (partitions.isEmpty()) { - LOG.info( - "Skipping table {}: " - + "table is partitioned, but no eligible partitions found => nothing to export.", - table.getTableName()); - return true; - } - return false; - } - - @Override - public boolean shouldSkipPartition( - BigQueryTable.Builder table, BigQueryTablePartition partition) { - if (maxLastModifiedTime != null - // BigQuery timestamps are in microseconds so / 1000. - && maxLastModifiedTime.isBefore(partition.getLastModificationTime() / 1000)) { - return true; - } - if (includePartitions != null && !includePartitions.matches(partition.getPartitionName())) { - LOG.info( - "Skipping partition {} not matching regexp: {}", - partition.getPartitionName(), - includePartitions.pattern()); - return true; - } - // Check if target file already exists - String expectedTargetPath = - String.format( - "%s/%s/%s_pid=%s/output-%s-%s%s", - targetRootPath, - table.getTableName(), - table.getPartitioningColumn(), - partition.getPartitionName(), - table.getTableName(), - partition.getPartitionName(), - fileSuffix); - if (existingTargetFiles.contains(expectedTargetPath)) { - return shouldSkipFile(table.getTableName(), partition.getPartitionName()); - } - return false; - } - } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java new file mode 100644 index 0000000000..67347e5598 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2021 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.v2.utils; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; +import com.google.cloud.teleport.v2.values.BigQueryTable; +import com.google.cloud.teleport.v2.values.BigQueryTablePartition; +import com.google.common.base.Splitter; +import com.google.re2j.Pattern; +import java.time.Duration; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.joda.time.Instant; +import org.joda.time.format.ISODateTimeFormat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Filter to exclude unwanted tables and partitions. */ +public class DataplexBigQueryToGcsFilter implements BigQueryMetadataLoader.Filter { + private static final Logger LOG = LoggerFactory.getLogger(DataplexBigQueryToGcsFilter.class); + private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); + + private final Instant maxLastModifiedTime; + private final Set includeTables; + private final Pattern includePartitions; + private final String targetRootPath; + private final String writeDisposition; + private final String fileSuffix; + private final List existingTargetFiles; + + public DataplexBigQueryToGcsFilter( + DataplexBigQueryToGcsOptions options, + String targetRootPath, + List existingTargetFiles) { + String dateTime = options.getExportDataModifiedBeforeDateTime(); + if (dateTime != null && !dateTime.isEmpty()) { + if (dateTime.startsWith("-P") || dateTime.startsWith("-p")) { + this.maxLastModifiedTime = Instant.now().plus(Duration.parse(dateTime).toMillis()); + } else { + this.maxLastModifiedTime = + Instant.parse(dateTime, ISODateTimeFormat.dateOptionalTimeParser()); + } + } else { + this.maxLastModifiedTime = null; + } + + String tableRefs = options.getTableRefs(); + if (tableRefs != null && !tableRefs.isEmpty()) { + List tableRefList = SPLITTER.splitToList(tableRefs); + checkArgument( + !tableRefList.isEmpty(), + "Got an non-empty tableRefs param '%s', but couldn't parse it into a valid table list," + + " please check its format.", + tableRefs); + this.includeTables = new HashSet<>(tableRefList); + } else { + this.includeTables = null; + } + + String partitionRegExp = options.getPartitionIdRegExp(); + if (partitionRegExp != null && !partitionRegExp.isEmpty()) { + this.includePartitions = Pattern.compile(partitionRegExp); + } else { + this.includePartitions = null; + } + this.targetRootPath = targetRootPath; + this.writeDisposition = options.getWriteDisposition().getWriteDisposition(); + this.fileSuffix = options.getFileFormat().getFileSuffix(); + this.existingTargetFiles = existingTargetFiles; + } + + private boolean shouldSkipTableName(BigQueryTable.Builder table) { + if (includeTables != null && !includeTables.contains(table.getTableName())) { + return true; + } + return false; + } + + private boolean shouldSkipFile(String table, String partition) { + String identifier = partition == null ? table : table + "$" + partition; + switch (writeDisposition) { + case "FAIL": + throw new RuntimeException( + String.format( + "Target File exists for {}. Failing according to writeDisposition", identifier)); + case "SKIP": + LOG.info("Target File exists for {}. Skipping according to writeDisposition", identifier); + return true; + case "OVERWRITE": + LOG.info( + "Target File exists for {}. Overwriting according to writeDisposition", identifier); + return false; + default: + throw new UnsupportedOperationException( + writeDisposition + " writeDisposition not implemented"); + } + } + + @Override + public boolean shouldSkipUnpartitionedTable(BigQueryTable.Builder table) { + if (shouldSkipTableName(table)) { + return true; + } + // Check the last modified time only for NOT partitioned table. + // If a table is partitioned, we check the last modified time on partition level only. + if (maxLastModifiedTime != null + // BigQuery timestamps are in microseconds so / 1000. + && maxLastModifiedTime.isBefore(table.getLastModificationTime() / 1000)) { + return true; + } + // Check if the target file already exists + String expectedTargetPath = + String.format( + "%s/%s/output-%s%s", + targetRootPath, table.getTableName(), table.getTableName(), fileSuffix); + if (existingTargetFiles.contains(expectedTargetPath)) { + return shouldSkipFile(table.getTableName(), null); + } + return false; + } + + @Override + public boolean shouldSkipPartitionedTable( + BigQueryTable.Builder table, List partitions) { + if (shouldSkipTableName(table)) { + return true; + } + if (partitions.isEmpty()) { + LOG.info( + "Skipping table {}: " + + "table is partitioned, but no eligible partitions found => nothing to export.", + table.getTableName()); + return true; + } + return false; + } + + @Override + public boolean shouldSkipPartition( + BigQueryTable.Builder table, BigQueryTablePartition partition) { + if (maxLastModifiedTime != null + // BigQuery timestamps are in microseconds so / 1000. + && maxLastModifiedTime.isBefore(partition.getLastModificationTime() / 1000)) { + return true; + } + if (includePartitions != null && !includePartitions.matches(partition.getPartitionName())) { + LOG.info( + "Skipping partition {} not matching regexp: {}", + partition.getPartitionName(), + includePartitions.pattern()); + return true; + } + // Check if target file already exists + String expectedTargetPath = + String.format( + "%s/%s/%s_pid=%s/output-%s-%s%s", + targetRootPath, + table.getTableName(), + table.getPartitioningColumn(), + partition.getPartitionName(), + table.getTableName(), + partition.getPartitionName(), + fileSuffix); + if (existingTargetFiles.contains(expectedTargetPath)) { + return shouldSkipFile(table.getTableName(), partition.getPartitionName()); + } + return false; + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java index 39da9a2945..5e894518d8 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java @@ -33,23 +33,33 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.api.services.bigquery.model.TimePartitioning; import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.DatasetId; import com.google.cloud.bigquery.EmptyTableResult; +import com.google.cloud.bigquery.FieldValue; +import com.google.cloud.bigquery.FieldValueList; import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.bigquery.TableId; -import com.google.cloud.teleport.v2.templates.DataplexBigQueryToGcs.DataplexBigQueryToGcsOptions; +import com.google.cloud.bigquery.TableResult; +import com.google.cloud.bigquery.storage.v1beta1.BigQueryStorageClient; +import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; +import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; +import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.common.collect.ImmutableList; import java.io.ByteArrayOutputStream; import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileWriter; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Scanner; import org.apache.avro.Schema; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.KvCoder; @@ -86,6 +96,7 @@ public class DataplexBigQueryToGcsTest { private static final String PROJECT = "test-project1"; private static final String DATASET = "test-dataset1"; + private static final int MAX_PARALLEL_REQUESTS = 5; @Rule public final MockitoRule mockito = MockitoJUnit.rule(); @@ -94,6 +105,9 @@ public class DataplexBigQueryToGcsTest { @Rule public final TestPipeline testPipeline = TestPipeline.create(); @Mock private static BigQuery bqMock; + @Mock private static BigQueryStorageClient bqsMock; + @Mock private static TableResult tableResultMock; + private BigQueryMetadataLoader metadataLoader; private BigQueryServices bqFakeServices; private CustomFakeJobService fakeJobService; private FakeDatasetService fakeDatasetService; @@ -202,14 +216,23 @@ public void setUp() throws InterruptedException, IOException { new FakeBigQueryServices() .withJobService(fakeJobService) .withDatasetService(fakeDatasetService); - - when(bqMock.query(any())).thenReturn(new EmptyTableResult(null)); + metadataLoader = new BigQueryMetadataLoader(bqMock, bqsMock, MAX_PARALLEL_REQUESTS); + + List fieldValueList = + Arrays.asList( + FieldValue.of(FieldValue.Attribute.RECORD, "unpartitioned_table"), + FieldValue.of(FieldValue.Attribute.RECORD, "0"), + FieldValue.of(FieldValue.Attribute.RECORD, null)); + when(tableResultMock.iterateAll()).thenReturn(Arrays.asList(FieldValueList.of(fieldValueList))); + when(bqMock.query(any())).thenReturn(tableResultMock); when(bqMock.delete(any(TableId.class))).thenReturn(true); } @Test @Category(NeedsRunner.class) public void testE2E_mainPathWithAllStepsEnabled() throws Exception { + when(bqMock.query(any())).thenReturn(new EmptyTableResult(null)); + insertTableData("unpartitioned_table", defaultRecords); insertPartitionData("partitioned_table", "p1", Arrays.copyOfRange(defaultRecords, 0, 2)); insertPartitionData("partitioned_table", "p2", Arrays.copyOfRange(defaultRecords, 2, 5)); @@ -323,6 +346,39 @@ public void testE2E_withDeleteSourceDataDisabled_doesntTruncateData() throws Exc verifyNoMoreInteractions(bqMock); } + @Test(expected = Exception.class) + @Category(NeedsRunner.class) + public void testE2E_withTargetStrategyFail_throwsException() throws Exception { + options.setFileFormat(FileFormat.PARQUET); + options.setWriteDisposition(WriteDisposition.FAIL); + File outputFile = + writeOutputFile("unpartitioned_table", "output-unpartitioned_table.parquet", "Test data"); + + Pipeline p = + DataplexBigQueryToGcs.buildPipeline( + options, metadataLoader, outDir.getAbsolutePath(), DatasetId.of(PROJECT, DATASET)); + p.run(); + testPipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testE2E_withTargetStrategySkip_skipsTable() throws Exception { + options.setFileFormat(FileFormat.PARQUET); + options.setWriteDisposition(WriteDisposition.SKIP); + File outputFile = + writeOutputFile("unpartitioned_table", "output-unpartitioned_table.parquet", "Test data"); + + Pipeline p = + DataplexBigQueryToGcs.buildPipeline( + options, metadataLoader, outDir.getAbsolutePath(), DatasetId.of(PROJECT, DATASET)); + p.run(); + testPipeline.run(); + // Checking to see if the file was skipped and data was not overwritten + + assertThat(readFirstLine(outputFile)).isEqualTo("Test data"); + } + @Test public void testGetFilesInDirectory_withValidPath_returnsPathsOfFilesInDirectory() throws Exception { @@ -339,6 +395,23 @@ public void testGetFilesInDirectory_withValidPath_returnsPathsOfFilesInDirectory assertThat(files.size()).isEqualTo(2); } + private String readFirstLine(File outputFile) throws FileNotFoundException { + Scanner fileReader = new Scanner(outputFile); + String result = fileReader.nextLine(); + fileReader.close(); + return result; + } + + private File writeOutputFile(String folderName, String filename, String data) throws IOException { + File outputDir = tmpDir.newFolder("out", folderName); + File outputFile = new File(outputDir.getAbsolutePath() + "/" + filename); + outputFile.createNewFile(); + FileWriter writer = new FileWriter(outputFile); + writer.write(data); + writer.close(); + return outputFile; + } + private void insertTableData(String tableName, TableRow... records) throws Exception { fakeDatasetService.insertAll( tableByName.get(tableName).toTableReference(), Arrays.asList(records), null); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java similarity index 90% rename from v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java rename to v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java index f978502694..7e6b0bce63 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsFilterTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java @@ -13,12 +13,11 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.templates; +package com.google.cloud.teleport.v2.utils; import static com.google.common.truth.Truth.assertThat; -import com.google.cloud.teleport.v2.templates.DataplexBigQueryToGcs.DataplexBigQueryToGcsOptions; -import com.google.cloud.teleport.v2.templates.DataplexBigQueryToGcs.MetadataFilter; +import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader.Filter; @@ -37,7 +36,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Unit tests for {@link DataplexBigQueryToGcs.MetadataFilter}. */ +/** Unit tests for {@link DataplexBigQueryToGcsFilter}. */ @RunWith(JUnit4.class) public class DataplexBigQueryToGcsFilterTest { public static final Long TS_MICROS_2021_01_01_15_00_00_UTC = 1609513200000000L; @@ -57,7 +56,7 @@ public void test_whenNoFilterOptions_filterAcceptsAllTablesAndPartitions() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(t)).isFalse(); assertThat(f.shouldSkipPartitionedTable(t, Collections.singletonList(p))).isFalse(); @@ -74,7 +73,7 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { options.setTableRefs("includedTable1,includedTable2"); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(includedTable1)).isFalse(); assertThat(f.shouldSkipUnpartitionedTable(includedTable2)).isFalse(); @@ -93,7 +92,7 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { @Test(expected = IllegalArgumentException.class) public void test_whenTableRefsIsInvalid_throwsException() { options.setTableRefs(","); - new MetadataFilter(options, "", new ArrayList()); + new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); } @Test @@ -117,7 +116,7 @@ public void test_whenBeforeDateSet_filterExcludesTablesAndPartitions() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); @@ -141,7 +140,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -149,7 +148,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { // Should be the same as 15:00 UTC: options.setExportDataModifiedBeforeDateTime("2021-01-01T14:00:00-01:00"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -157,7 +156,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { // Should be the same as 15:00 UTC: options.setExportDataModifiedBeforeDateTime("2021-01-01T17:00:00+02:00"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -166,7 +165,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { // 14:00 UTC is 1 hour is earlier that both table's last modified time // (14:59:59.999 and 15:00:00.001 UTC). Expecting both to be skipped. options.setExportDataModifiedBeforeDateTime("2021-01-01T14:00:00Z"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); } @@ -183,7 +182,7 @@ public void test_whenBeforeDateHasNoTime_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("2021-02-15"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -199,7 +198,7 @@ public void test_whenBeforeDateIs1DayDuration_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("-P1D"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); } @@ -215,7 +214,7 @@ public void test_whenBeforeDateIs1Day3HoursDuration_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("-p1dt3h"); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); } @@ -225,7 +224,7 @@ public void test_whenPartitionedTableHasNoPartitions_filterExcludesTable() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new MetadataFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); assertThat(f.shouldSkipPartitionedTable(table(), Collections.emptyList())).isTrue(); } @@ -245,7 +244,7 @@ public void test_whenTargetFileExistsWithWriteDisposionSKIP_filterExcludesTables String targetfilePathPartitionedTable = String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); Filter f = - new MetadataFilter( + new DataplexBigQueryToGcsFilter( options, targetRootPath, new ArrayList() { @@ -274,7 +273,7 @@ public void test_whenTargetFileExistsWithWriteDisposionOverwrite_filterAcceptsTa String targetfilePathPartitionedTable = String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); Filter f = - new MetadataFilter( + new DataplexBigQueryToGcsFilter( options, targetRootPath, new ArrayList() { @@ -303,7 +302,7 @@ public void test_whenTargetFileExistsWithWriteDisposionFail_filterAcceptsTables( String targetfilePathPartitionedTable = String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); Filter f = - new MetadataFilter( + new DataplexBigQueryToGcsFilter( options, targetRootPath, new ArrayList() { From e8a2aa9c820554cc08ef3f5626025b6b81609405 Mon Sep 17 00:00:00 2001 From: anikin Date: Thu, 30 Dec 2021 09:56:57 -0800 Subject: [PATCH 010/145] Add spannerProjectId param to the Text Files to Cloud Spanner template. PiperOrigin-RevId: 418972160 --- .../google/cloud/teleport/spanner/TextImportPipeline.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java index cf63b0ce4c..cb5b2bf571 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java @@ -165,6 +165,11 @@ public interface Options extends PipelineOptions { void setWaitUntilFinish(boolean value); + @Description("GCP Project Id of where the Spanner table lives.") + ValueProvider getSpannerProjectId(); + + void setSpannerProjectId(ValueProvider value); + @Description("The spanner priority. --spannerPriority must be one of:[HIGH,MEDIUM,LOW]") ValueProvider getSpannerPriority(); @@ -179,6 +184,7 @@ public static void main(String[] args) { SpannerConfig spannerConfig = SpannerConfig.create() + .withProjectId(options.getSpannerProjectId()) .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) .withDatabaseId(options.getDatabaseId()) From c57187e036dccdf537b939d2b68bead0b3068282 Mon Sep 17 00:00:00 2001 From: anikin Date: Thu, 30 Dec 2021 12:58:46 -0800 Subject: [PATCH 011/145] Dataplex Tiering template: add an option to retain the partition ID. Also moved the logic of generating the sub directory names in the target file path to BigQueryToGcsDirectoryNaming class. BigQueryTableToGcsTransform and DataplexBigQueryToGcsFilter now use BigQueryToGcsDirectoryNaming + BigQueryToGcsFileNaming to generate target file paths (these have to match, otherwise the Filter won't skip the files generated by the Transform). A few minor bugfixes and test improvements. PiperOrigin-RevId: 418996305 --- .../options/DataplexBigQueryToGcsOptions.java | 11 + .../v2/templates/DataplexBigQueryToGcs.java | 13 +- .../BigQueryTableToGcsTransform.java | 55 +++-- .../utils/BigQueryToGcsDirectoryNaming.java | 41 ++++ .../v2/utils/DataplexBigQueryToGcsFilter.java | 81 +++++--- .../cloud/teleport/v2/utils/Schemas.java | 42 ++++ .../templates/DataplexBigQueryToGcsTest.java | 194 ++++++++++++++++-- .../DataplexBigQueryToGcsFilterTest.java | 79 +++---- .../cloud/teleport/v2/utils/SchemasTest.java | 21 ++ .../org.mockito.plugins.MockMaker | 2 + 10 files changed, 419 insertions(+), 120 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsDirectoryNaming.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java index e11ccb31a7..466e1a23ac 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java @@ -118,4 +118,15 @@ public interface DataplexBigQueryToGcsOptions WriteDisposition getWriteDisposition(); void setWriteDisposition(WriteDisposition writeDisposition); + + @Description( + "Due to a BigQuery limitation, it's not possible to have a partitioned external table with" + + " the partition key (in the file path) having the same name as one of the columns in" + + " the file. If enforceSamePartitionKey is true (the default), the partition key of" + + " the target file will be set to the original partition column name and the column in" + + " the file will be renamed. If false, it's the partition key that will be renamed.") + @Default.Boolean(true) + Boolean getEnforceSamePartitionKey(); + + void setEnforceSamePartitionKey(Boolean enforceSamePartitionKey); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index f5320e69c3..5b73ca4936 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -60,6 +60,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -142,7 +143,6 @@ private static Pipeline setUpPipeline( DataplexAssetResourceSpec.BIGQUERY_DATASET); String targetRootPath = "gs://" + gcsResource; - List existingTargetFiles = getFilesInDirectory(targetRootPath); DatasetId datasetId = BigQueryUtils.parseDatasetUrn(bqResource); BigQueryMetadataLoader metadataLoader = @@ -172,7 +172,7 @@ static Pipeline buildPipeline( List tables = metadataLoader.loadDatasetMetadata( datasetId, - new DataplexBigQueryToGcsFilter(options, targetRootPath, existingTargetFiles)); + new DataplexBigQueryToGcsFilter(options, existingTargetFiles)); LOG.info("Loaded {} table(s).", tables.size()); if (!tables.isEmpty()) { @@ -203,7 +203,8 @@ static void transformPipeline( table, targetRootPath, options.getFileFormat(), - options.getFileCompression()) + options.getFileCompression(), + options.getEnforceSamePartitionKey()) .withTestServices(testBqServices)) .apply( String.format("AttachTableKeys-%s", table.getTableName()), @@ -262,17 +263,17 @@ private static String resolveAsset( @VisibleForTesting static List getFilesInDirectory(String path) { try { - String pattern = String.format("%s/**", path); - MatchResult result = FileSystems.match(pattern, EmptyMatchTreatment.ALLOW); + String pathPrefix = path + "/"; + MatchResult result = FileSystems.match(pathPrefix + "**", EmptyMatchTreatment.ALLOW); List fileNames = result.metadata().stream() .map(MatchResult.Metadata::resourceId) .map(ResourceId::toString) + .map(s -> StringUtils.removeStart(s, pathPrefix)) .collect(toList()); LOG.info("{} file(s) found in directory {}", fileNames.size(), path); return fileNames; } catch (Exception e) { - LOG.error("Exception thrown while getting output files in gcs resource."); throw new RuntimeException(e); } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java index 9e483b8f57..ccbc877210 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java @@ -15,7 +15,9 @@ */ package com.google.cloud.teleport.v2.transforms; +import com.google.cloud.teleport.v2.utils.BigQueryToGcsDirectoryNaming; import com.google.cloud.teleport.v2.utils.BigQueryToGcsFileNaming; +import com.google.cloud.teleport.v2.utils.Schemas; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexCompression; @@ -58,41 +60,63 @@ public class BigQueryTableToGcsTransform extends PTransform>> { + private static final String PARTITION_COLUMN_RENAME_SUFFIX = "_pkey"; + private final BigQueryTable table; private final FileFormat outputFileFormat; private final DataplexCompression outputFileCompression; private final String targetRootPath; + private final boolean enforceSamePartitionKey; private transient BigQueryServices testServices; public BigQueryTableToGcsTransform( BigQueryTable table, String targetRootPath, FileFormat outputFileFormat, - DataplexCompression outputFileCompression) { + DataplexCompression outputFileCompression, + boolean enforceSamePartitionKey) { this.table = table; this.targetRootPath = targetRootPath; this.outputFileFormat = outputFileFormat; this.outputFileCompression = outputFileCompression; + this.enforceSamePartitionKey = enforceSamePartitionKey; } @Override public PCollection> expand(PBegin begin) { - Schema schema = table.getSchema(); + Schema targetFileSchema = table.getSchema(); + if (table.isPartitioned() && enforceSamePartitionKey) { + // Apart from renaming the field in the schema we don't need to anything else (e.g. replace + // the field in the actual GenericRecord being processed) because writers write fields + // to the file based on their numeric position, not their name. + targetFileSchema = + Schemas.renameAvroField( + targetFileSchema, + table.getPartitioningColumn(), + table.getPartitioningColumn() + PARTITION_COLUMN_RENAME_SUFFIX); + } + Sink sink; switch (outputFileFormat) { case PARQUET: - sink = ParquetIO.sink(schema).withCompressionCodec(outputFileCompression.getParquetCodec()); + sink = + ParquetIO.sink(targetFileSchema) + .withCompressionCodec(outputFileCompression.getParquetCodec()); break; case AVRO: - sink = AvroIO.sink(schema).withCodec(outputFileCompression.getAvroCodec()); + sink = + AvroIO.sink(targetFileSchema) + .withCodec(outputFileCompression.getAvroCodec()); break; default: throw new UnsupportedOperationException( "Output format is not implemented: " + outputFileFormat); } + BigQueryToGcsDirectoryNaming dn = new BigQueryToGcsDirectoryNaming(enforceSamePartitionKey); + if (!table.isPartitioned()) { - return transformTable(begin, sink); + return transformTable(begin, sink, dn); } if (table.getPartitions() == null || table.getPartitions().isEmpty()) { throw new IllegalStateException( @@ -102,14 +126,16 @@ public PCollection> expand(PBegin begin) { } List>> collections = new ArrayList<>(); - table.getPartitions().forEach(p -> collections.add(transformPartition(begin, sink, p))); + table.getPartitions().forEach(p -> collections.add(transformPartition(begin, sink, p, dn))); return PCollectionList.of(collections) .apply(tableNodeName("FlattenPartitionResults"), Flatten.pCollections()); } private PCollection> transformTable( - PBegin begin, Sink sink) { - String targetPath = String.format("%s/%s", targetRootPath, table.getTableName()); + PBegin begin, Sink sink, BigQueryToGcsDirectoryNaming directoryNaming) { + String targetPath = + String.format( + "%s/%s", targetRootPath, directoryNaming.getTableDirectory(table.getTableName())); return begin .apply(tableNodeName("Read"), getDefaultRead().from(table.toTableReference())) @@ -132,7 +158,11 @@ private PCollection> transformTable( } private PCollection> transformPartition( - PBegin begin, Sink sink, BigQueryTablePartition partition) { + PBegin begin, + Sink sink, + BigQueryTablePartition partition, + BigQueryToGcsDirectoryNaming directoryNaming) { + String sql = String.format( "select * from [%s.%s.%s$%s]", @@ -143,11 +173,10 @@ private PCollection> transformPartition( String targetPath = String.format( - "%s/%s/%s_pid=%s", + "%s/%s", targetRootPath, - table.getTableName(), - table.getPartitioningColumn(), - partition.getPartitionName()); + directoryNaming.getPartitionDirectory( + table.getTableName(), partition.getPartitionName(), table.getPartitioningColumn())); return begin .apply(partitionNodeName("Read", partition), getDefaultRead().fromQuery(sql)) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsDirectoryNaming.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsDirectoryNaming.java new file mode 100644 index 0000000000..60a173e4a6 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryToGcsDirectoryNaming.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2021 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.v2.utils; + +/** Generates subdirectory names used for storing BigQuery tables/partitions in GCS. */ +public class BigQueryToGcsDirectoryNaming { + private static final String PARTITION_ID_RENAME_SUFFIX = "_pid"; + + private final boolean enforceSamePartitionKey; + + public BigQueryToGcsDirectoryNaming(boolean enforceSamePartitionKey) { + this.enforceSamePartitionKey = enforceSamePartitionKey; + } + + public String getTableDirectory(String tableName) { + return tableName; + } + + public String getPartitionDirectory( + String tableName, String partitionName, String partitioningColumn) { + return String.format( + "%s/%s%s=%s", + tableName, + partitioningColumn, + enforceSamePartitionKey ? "" : PARTITION_ID_RENAME_SUFFIX, + partitionName); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java index 67347e5598..ea7aeba14e 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java @@ -39,15 +39,13 @@ public class DataplexBigQueryToGcsFilter implements BigQueryMetadataLoader.Filte private final Instant maxLastModifiedTime; private final Set includeTables; private final Pattern includePartitions; - private final String targetRootPath; private final String writeDisposition; private final String fileSuffix; private final List existingTargetFiles; + private final BigQueryToGcsDirectoryNaming directoryNaming; public DataplexBigQueryToGcsFilter( - DataplexBigQueryToGcsOptions options, - String targetRootPath, - List existingTargetFiles) { + DataplexBigQueryToGcsOptions options, List existingTargetFiles) { String dateTime = options.getExportDataModifiedBeforeDateTime(); if (dateTime != null && !dateTime.isEmpty()) { if (dateTime.startsWith("-P") || dateTime.startsWith("-p")) { @@ -79,10 +77,11 @@ public DataplexBigQueryToGcsFilter( } else { this.includePartitions = null; } - this.targetRootPath = targetRootPath; + this.writeDisposition = options.getWriteDisposition().getWriteDisposition(); this.fileSuffix = options.getFileFormat().getFileSuffix(); this.existingTargetFiles = existingTargetFiles; + this.directoryNaming = new BigQueryToGcsDirectoryNaming(options.getEnforceSamePartitionKey()); } private boolean shouldSkipTableName(BigQueryTable.Builder table) { @@ -92,19 +91,27 @@ private boolean shouldSkipTableName(BigQueryTable.Builder table) { return false; } - private boolean shouldSkipFile(String table, String partition) { + private boolean shouldSkipFile(String table, String partition, String path) { String identifier = partition == null ? table : table + "$" + partition; switch (writeDisposition) { case "FAIL": - throw new RuntimeException( + throw new WriteDispositionException( String.format( - "Target File exists for {}. Failing according to writeDisposition", identifier)); + "Target File %s exists for %s. Failing according to writeDisposition = %s.", + path, identifier, writeDisposition)); case "SKIP": - LOG.info("Target File exists for {}. Skipping according to writeDisposition", identifier); + LOG.info( + "Target File {} exists for {}. Skipping according to writeDisposition = {}.", + path, + identifier, + writeDisposition); return true; case "OVERWRITE": LOG.info( - "Target File exists for {}. Overwriting according to writeDisposition", identifier); + "Target File {} exists for {}. Overwriting according to writeDisposition = {}.", + path, + identifier, + writeDisposition); return false; default: throw new UnsupportedOperationException( @@ -124,13 +131,10 @@ public boolean shouldSkipUnpartitionedTable(BigQueryTable.Builder table) { && maxLastModifiedTime.isBefore(table.getLastModificationTime() / 1000)) { return true; } - // Check if the target file already exists - String expectedTargetPath = - String.format( - "%s/%s/output-%s%s", - targetRootPath, table.getTableName(), table.getTableName(), fileSuffix); + // Check if the target file already exists. + String expectedTargetPath = tableTargetFileName(table); if (existingTargetFiles.contains(expectedTargetPath)) { - return shouldSkipFile(table.getTableName(), null); + return shouldSkipFile(table.getTableName(), null, expectedTargetPath); } return false; } @@ -166,20 +170,41 @@ public boolean shouldSkipPartition( includePartitions.pattern()); return true; } - // Check if target file already exists - String expectedTargetPath = - String.format( - "%s/%s/%s_pid=%s/output-%s-%s%s", - targetRootPath, - table.getTableName(), - table.getPartitioningColumn(), - partition.getPartitionName(), - table.getTableName(), - partition.getPartitionName(), - fileSuffix); + // Check if target file already exists. + String expectedTargetPath = partitionTargetFileName(table, partition); if (existingTargetFiles.contains(expectedTargetPath)) { - return shouldSkipFile(table.getTableName(), partition.getPartitionName()); + return shouldSkipFile(table.getTableName(), partition.getPartitionName(), expectedTargetPath); } return false; } + + public String tableTargetFileName(BigQueryTable.Builder table) { + String dirName = directoryNaming.getTableDirectory(table.getTableName()); + String fileName = + new BigQueryToGcsFileNaming(fileSuffix, table.getTableName()) + .getFilename(null, null, 0, 0, null); + return dirName + "/" + fileName; + } + + public String partitionTargetFileName( + BigQueryTable.Builder table, BigQueryTablePartition partition) { + String dirName = + directoryNaming.getPartitionDirectory( + table.getTableName(), partition.getPartitionName(), table.getPartitioningColumn()); + String fileName = + new BigQueryToGcsFileNaming(fileSuffix, table.getTableName(), partition.getPartitionName()) + .getFilename(null, null, 0, 0, null); + return dirName + "/" + fileName; + } + + /** + * Thrown if {@link + * com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition + * WriteDisposition} is set to {@code FAIL} and a target file exists. + */ + public static class WriteDispositionException extends RuntimeException { + public WriteDispositionException(String message) { + super(message); + } + } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java index 91e9e11379..585ef23525 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java @@ -17,18 +17,24 @@ import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1Schema; import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1SchemaSchemaField; +import java.io.File; +import java.io.IOException; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.ArrayList; import java.util.List; import javax.sql.DataSource; import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; import org.apache.avro.SchemaBuilder; import org.apache.avro.SchemaBuilder.BaseTypeBuilder; import org.apache.avro.SchemaBuilder.FieldAssembler; import org.apache.avro.SchemaBuilder.RecordBuilder; import org.apache.avro.SchemaBuilder.TypeBuilder; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericDatumReader; import org.apache.beam.sdk.io.jdbc.BeamSchemaUtil; import org.apache.beam.sdk.schemas.utils.AvroUtils; @@ -144,4 +150,40 @@ private static Schema dataplexFieldsToAvro( } return avroFieldAssembler.endRecord(); } + + /** + * Returns a new Avro schema where {@code fromFieldName} in the {@code originalSchema} is renamed + * to {@code toFieldName}. + * + *

The old field name will be added as a field alias. + */ + public static Schema renameAvroField( + Schema originalSchema, String fromFieldName, String toFieldName) { + List fields = new ArrayList<>(originalSchema.getFields().size()); + + for (Field f : originalSchema.getFields()) { + Field newField; + if (f.name().equals(fromFieldName)) { + newField = new Field(toFieldName, f.schema(), f.doc(), f.defaultVal(), f.order()); + newField.addAlias(fromFieldName); + } else { + newField = new Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order()); + } + fields.add(newField); + } + + return Schema.createRecord( + originalSchema.getName(), + originalSchema.getDoc(), + originalSchema.getNamespace(), + false, + fields); + } + + public static Schema avroSchemaFromDataFile(String path) throws IOException { + try (DataFileReader reader = + new DataFileReader<>(new File(path), new GenericDatumReader<>())) { + return reader.getSchema(); + } + } } diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java index 5e894518d8..83c1b055ae 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java @@ -16,6 +16,9 @@ package com.google.cloud.teleport.v2.templates; import static com.google.common.truth.Truth.assertThat; +import static com.google.re2j.Pattern.CASE_INSENSITIVE; +import static com.google.re2j.Pattern.DOTALL; +import static org.junit.Assert.fail; import static org.mockito.Mockito.any; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -40,20 +43,26 @@ import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableResult; +import com.google.cloud.bigquery.storage.v1beta1.AvroProto.AvroSchema; import com.google.cloud.bigquery.storage.v1beta1.BigQueryStorageClient; +import com.google.cloud.bigquery.storage.v1beta1.Storage.ReadSession; import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader; +import com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.Schemas; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.common.collect.ImmutableList; +import com.google.re2j.Pattern; import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileNotFoundException; import java.io.FileWriter; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -66,6 +75,7 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices; import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; @@ -99,14 +109,16 @@ public class DataplexBigQueryToGcsTest { private static final int MAX_PARALLEL_REQUESTS = 5; @Rule public final MockitoRule mockito = MockitoJUnit.rule(); - @Rule public final TemporaryFolder tmpDir = new TemporaryFolder(); - @Rule public final TestPipeline testPipeline = TestPipeline.create(); + // bqMock has to be static, otherwise it won't be serialized properly when passed to + // DeleteBigQueryDataFn#withTestBqClientFactory. @Mock private static BigQuery bqMock; - @Mock private static BigQueryStorageClient bqsMock; - @Mock private static TableResult tableResultMock; + + @Mock private BigQueryStorageClient bqsMock; + @Mock private TableResult tableResultMock; + private BigQueryMetadataLoader metadataLoader; private BigQueryServices bqFakeServices; private CustomFakeJobService fakeJobService; @@ -124,6 +136,7 @@ public void setUp() throws InterruptedException, IOException { options = TestPipeline.testingPipelineOptions().as(DataplexBigQueryToGcsOptions.class); options.setProject(PROJECT); options.setUpdateDataplexMetadata(true); + options.setEnforceSamePartitionKey(false); // Required when using BigQueryIO.withMethod(EXPORT). options.setTempLocation(tmpDir.newFolder("bqTmp").getAbsolutePath()); @@ -216,16 +229,18 @@ public void setUp() throws InterruptedException, IOException { new FakeBigQueryServices() .withJobService(fakeJobService) .withDatasetService(fakeDatasetService); - metadataLoader = new BigQueryMetadataLoader(bqMock, bqsMock, MAX_PARALLEL_REQUESTS); - List fieldValueList = - Arrays.asList( - FieldValue.of(FieldValue.Attribute.RECORD, "unpartitioned_table"), - FieldValue.of(FieldValue.Attribute.RECORD, "0"), - FieldValue.of(FieldValue.Attribute.RECORD, null)); - when(tableResultMock.iterateAll()).thenReturn(Arrays.asList(FieldValueList.of(fieldValueList))); + when(tableResultMock.iterateAll()) + .thenReturn(Collections.singleton(fields("unpartitioned_table", "0", null))); when(bqMock.query(any())).thenReturn(tableResultMock); when(bqMock.delete(any(TableId.class))).thenReturn(true); + when(bqsMock.createReadSession(any())) + .thenReturn( + ReadSession.newBuilder() + .setAvroSchema(AvroSchema.newBuilder().setSchema(avroSchema.toString())) + .build()); + + metadataLoader = new BigQueryMetadataLoader(bqMock, bqsMock, MAX_PARALLEL_REQUESTS); } @Test @@ -346,19 +361,149 @@ public void testE2E_withDeleteSourceDataDisabled_doesntTruncateData() throws Exc verifyNoMoreInteractions(bqMock); } - @Test(expected = Exception.class) + @Test @Category(NeedsRunner.class) + public void testE2E_withEnforceSamePartitionKeyEnabled_producesRenamedColumns() throws Exception { + options.setEnforceSamePartitionKey(true); + options.setFileFormat(FileFormat.AVRO); + + insertPartitionData("partitioned_table", "p1", Arrays.copyOfRange(defaultRecords, 0, 2)); + insertPartitionData("partitioned_table", "p2", Arrays.copyOfRange(defaultRecords, 2, 5)); + + runTransform("partitioned_table"); + + Schema targetFileSchema = + Schemas.avroSchemaFromDataFile( + outDir.getAbsolutePath() + "/partitioned_table/ts=p1/output-partitioned_table-p1.avro"); + + // We extract Avro schema from the target data file and use it below instead of the manually + // created avroSchema (used in other tests)) to double-check the schema written to the file + // has the renamed ts_pkey column name and not the original "ts" name. + // Otherwise, AvroIO will just read the data file using whatever schema we manually provide + // and using the field names *we* provide and not those written to the file (it still works + // if the number of fields and their order/type match). + + PCollection actualRecords1 = + testPipeline + .apply( + "readP1", + AvroIO.readGenericRecords(targetFileSchema) + .from( + outDir.getAbsolutePath() + + "/partitioned_table/ts=p1/output-partitioned_table-p1.avro")) + .apply("mapP1", MapElements.into(TypeDescriptors.strings()).via(Object::toString)); + PCollection actualRecords2 = + testPipeline + .apply( + "readP2", + AvroIO.readGenericRecords(targetFileSchema) + .from( + outDir.getAbsolutePath() + + "/partitioned_table/ts=p2/output-partitioned_table-p2.avro")) + .apply("mapP2", MapElements.into(TypeDescriptors.strings()).via(Object::toString)); + + // Column "ts" should've been renamed to "ts_pkey": + + String[] expectedRecords1 = + new String[] { + "{\"ts_pkey\": 1, \"s1\": \"1001\", \"i1\": 2001}", + "{\"ts_pkey\": 2, \"s1\": \"1002\", \"i1\": 2002}" + }; + String[] expectedRecords2 = + new String[] { + "{\"ts_pkey\": 3, \"s1\": \"1003\", \"i1\": 2003}", + "{\"ts_pkey\": 4, \"s1\": \"1004\", \"i1\": null}", + "{\"ts_pkey\": 5, \"s1\": \"1005\", \"i1\": 2005}" + }; + + PAssert.that(actualRecords1).containsInAnyOrder(expectedRecords1); + PAssert.that(actualRecords2).containsInAnyOrder(expectedRecords2); + + // Verify that "_pid "is *not* appended in the *file path* if enforceSamePartitionKey = true, + // i.e. the below files should not have been created: + + PCollection actualNonExistingRecords = + testPipeline + .apply( + "readFiles", + AvroIO.readGenericRecords(targetFileSchema) + .withEmptyMatchTreatment(EmptyMatchTreatment.ALLOW) + .from(outDir.getAbsolutePath() + "/partitioned_table/ts_pid=p1/*.avro")) + .apply("mapFiles", MapElements.into(TypeDescriptors.strings()).via(Object::toString)); + + PAssert.that(actualNonExistingRecords).empty(); + + testPipeline.run(); + } + + @Test public void testE2E_withTargetStrategyFail_throwsException() throws Exception { options.setFileFormat(FileFormat.PARQUET); options.setWriteDisposition(WriteDisposition.FAIL); - File outputFile = - writeOutputFile("unpartitioned_table", "output-unpartitioned_table.parquet", "Test data"); - Pipeline p = - DataplexBigQueryToGcs.buildPipeline( - options, metadataLoader, outDir.getAbsolutePath(), DatasetId.of(PROJECT, DATASET)); - p.run(); - testPipeline.run(); + writeOutputFile("unpartitioned_table", "output-unpartitioned_table.parquet", "Test data"); + + try { + DataplexBigQueryToGcs.buildPipeline( + options, metadataLoader, outDir.getAbsolutePath(), DatasetId.of(PROJECT, DATASET)); + fail("Expected a WriteDispositionException"); + } catch (Exception e) { + assertThat(e).hasCauseThat().hasCauseThat().isInstanceOf(WriteDispositionException.class); + } + } + + private static final Pattern TABLE_QUERY_PATTERN = + Pattern.compile( + "select.*table_id.*last_modified_time.*partitioning_column", CASE_INSENSITIVE | DOTALL); + private static final Pattern PARTITION_QUERY_PATTERN = + Pattern.compile("select.*partition_id.*last_modified_time", CASE_INSENSITIVE | DOTALL); + /** + * Tests that the pipeline throws an exception if {@code writeDisposition = FAIL}, {@code + * enforceSamePartitionKey = true}, and one of the target files exist, when processing a + * partitioned table. + * + *

This is a special case because depending on the {@code enforceSamePartitionKey} param the + * generated file path can be different (for partitioned tables only!), so this verifies that + * {@link com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter + * DataplexBigQueryToGcsFilter} can find such files correctly. + */ + @Test + public void testE2E_withTargetStrategyFail_andEnforceSamePartitionKeyEnabled_throwsException() + throws Exception { + options.setFileFormat(FileFormat.PARQUET); + options.setWriteDisposition(WriteDisposition.FAIL); + options.setEnforceSamePartitionKey(true); + + writeOutputFile("partitioned_table/ts=p2", "output-partitioned_table-p2.parquet", "Test data"); + + when(bqMock.query(any())) + .then( + invocation -> { + Iterable result = null; + QueryJobConfiguration q = (QueryJobConfiguration) invocation.getArguments()[0]; + if (TABLE_QUERY_PATTERN.matcher(q.getQuery()).find()) { + result = Collections.singletonList(fields("partitioned_table", "0", "ts")); + } else if (PARTITION_QUERY_PATTERN.matcher(q.getQuery()).find()) { + result = Arrays.asList(fields("p1", "0"), fields("p2", "0")); + } + when(tableResultMock.iterateAll()).thenReturn(result); + return tableResultMock; + }); + + try { + DataplexBigQueryToGcs.buildPipeline( + options, metadataLoader, outDir.getAbsolutePath(), DatasetId.of(PROJECT, DATASET)); + fail("Expected a WriteDispositionException"); + } catch (Exception e) { + assertThat(e).hasCauseThat().hasCauseThat().isInstanceOf(WriteDispositionException.class); + assertThat(e) + .hasCauseThat() + .hasCauseThat() + .hasMessageThat() + .contains( + "Target File partitioned_table/ts=p2/output-partitioned_table-p2.parquet exists for" + + " partitioned_table$p2."); + } } @Test @@ -374,8 +519,8 @@ public void testE2E_withTargetStrategySkip_skipsTable() throws Exception { options, metadataLoader, outDir.getAbsolutePath(), DatasetId.of(PROJECT, DATASET)); p.run(); testPipeline.run(); - // Checking to see if the file was skipped and data was not overwritten + // Checking to see if the file was skipped and data was not overwritten assertThat(readFirstLine(outputFile)).isEqualTo("Test data"); } @@ -404,6 +549,7 @@ private String readFirstLine(File outputFile) throws FileNotFoundException { private File writeOutputFile(String folderName, String filename, String data) throws IOException { File outputDir = tmpDir.newFolder("out", folderName); + outputDir.mkdirs(); File outputFile = new File(outputDir.getAbsolutePath() + "/" + filename); outputFile.createNewFile(); FileWriter writer = new FileWriter(outputFile); @@ -489,4 +635,12 @@ public void startQueryJob(JobReference jobRef, JobConfigurationQuery query) { super.startQueryJob(jobRef, query); } } + + private static FieldValueList fields(Object... fieldValues) { + List list = new ArrayList<>(fieldValues.length); + for (Object fieldValue : fieldValues) { + list.add(FieldValue.of(FieldValue.Attribute.RECORD, fieldValue)); + } + return FieldValueList.of(list); + } } diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java index 7e6b0bce63..003169887b 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java @@ -21,6 +21,7 @@ import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader.Filter; +import com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter.WriteDispositionException; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import java.io.IOException; @@ -56,7 +57,7 @@ public void test_whenNoFilterOptions_filterAcceptsAllTablesAndPartitions() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(t)).isFalse(); assertThat(f.shouldSkipPartitionedTable(t, Collections.singletonList(p))).isFalse(); @@ -73,7 +74,7 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { options.setTableRefs("includedTable1,includedTable2"); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(includedTable1)).isFalse(); assertThat(f.shouldSkipUnpartitionedTable(includedTable2)).isFalse(); @@ -92,7 +93,7 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { @Test(expected = IllegalArgumentException.class) public void test_whenTableRefsIsInvalid_throwsException() { options.setTableRefs(","); - new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + new DataplexBigQueryToGcsFilter(options, new ArrayList()); } @Test @@ -116,7 +117,7 @@ public void test_whenBeforeDateSet_filterExcludesTablesAndPartitions() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); @@ -140,7 +141,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -148,7 +149,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { // Should be the same as 15:00 UTC: options.setExportDataModifiedBeforeDateTime("2021-01-01T14:00:00-01:00"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -156,7 +157,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { { // Should be the same as 15:00 UTC: options.setExportDataModifiedBeforeDateTime("2021-01-01T17:00:00+02:00"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -165,7 +166,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { // 14:00 UTC is 1 hour is earlier that both table's last modified time // (14:59:59.999 and 15:00:00.001 UTC). Expecting both to be skipped. options.setExportDataModifiedBeforeDateTime("2021-01-01T14:00:00Z"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); } @@ -182,7 +183,7 @@ public void test_whenBeforeDateHasNoTime_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("2021-02-15"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isFalse(); } @@ -198,7 +199,7 @@ public void test_whenBeforeDateIs1DayDuration_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("-P1D"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); } @@ -214,7 +215,7 @@ public void test_whenBeforeDateIs1Day3HoursDuration_dateParsedCorrectly() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime("-p1dt3h"); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipUnpartitionedTable(newerTable)).isTrue(); assertThat(f.shouldSkipUnpartitionedTable(olderTable)).isFalse(); } @@ -224,13 +225,13 @@ public void test_whenPartitionedTableHasNoPartitions_filterExcludesTable() { options.setTableRefs(null); options.setExportDataModifiedBeforeDateTime(null); - Filter f = new DataplexBigQueryToGcsFilter(options, "", new ArrayList()); + Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); assertThat(f.shouldSkipPartitionedTable(table(), Collections.emptyList())).isTrue(); } @Test - public void test_whenTargetFileExistsWithWriteDisposionSKIP_filterExcludesTables() { + public void test_whenTargetFileExistsWithWriteDispositionSKIP_filterExcludesTables() { BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); BigQueryTablePartition p = partition().setPartitionName("partition1").build(); @@ -238,28 +239,19 @@ public void test_whenTargetFileExistsWithWriteDisposionSKIP_filterExcludesTables options.setExportDataModifiedBeforeDateTime(null); options.setFileFormat(FileFormat.AVRO); options.setWriteDisposition(WriteDisposition.SKIP); - String targetRootPath = "gs://root/"; - String targetfilePathUnpartitionedTable = - String.format("%s/table1/output-table1.avro", targetRootPath); - String targetfilePathPartitionedTable = - String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); + Filter f = new DataplexBigQueryToGcsFilter( options, - targetRootPath, - new ArrayList() { - { - add(targetfilePathUnpartitionedTable); - add(targetfilePathPartitionedTable); - } - }); + Arrays.asList( + "table1/output-table1.avro", "table1/p2=partition1/output-table1-partition1.avro")); assertThat(f.shouldSkipUnpartitionedTable(t)).isTrue(); assertThat(f.shouldSkipPartition(t, p)).isTrue(); } @Test - public void test_whenTargetFileExistsWithWriteDisposionOverwrite_filterAcceptsTables() { + public void test_whenTargetFileExistsWithWriteDispositionOverwrite_filterAcceptsTables() { BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); BigQueryTablePartition p = partition().setPartitionName("partition1").build(); @@ -267,28 +259,19 @@ public void test_whenTargetFileExistsWithWriteDisposionOverwrite_filterAcceptsTa options.setExportDataModifiedBeforeDateTime(null); options.setFileFormat(FileFormat.AVRO); options.setWriteDisposition(WriteDisposition.OVERWRITE); - String targetRootPath = "gs://root/"; - String targetfilePathUnpartitionedTable = - String.format("%s/table1/output-table1.avro", targetRootPath); - String targetfilePathPartitionedTable = - String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); + Filter f = new DataplexBigQueryToGcsFilter( options, - targetRootPath, - new ArrayList() { - { - add(targetfilePathUnpartitionedTable); - add(targetfilePathPartitionedTable); - } - }); + Arrays.asList( + "table1/output-table1.avro", "table1/p2=partition1/output-table1-partition1.avro")); assertThat(f.shouldSkipUnpartitionedTable(t)).isFalse(); assertThat(f.shouldSkipPartition(t, p)).isFalse(); } - @Test(expected = RuntimeException.class) - public void test_whenTargetFileExistsWithWriteDisposionFail_filterAcceptsTables() { + @Test(expected = WriteDispositionException.class) + public void test_whenTargetFileExistsWithWriteDispositionFail_filterAcceptsTables() { BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); BigQueryTablePartition p = partition().setPartitionName("partition1").build(); @@ -296,21 +279,11 @@ public void test_whenTargetFileExistsWithWriteDisposionFail_filterAcceptsTables( options.setExportDataModifiedBeforeDateTime(null); options.setFileFormat(FileFormat.AVRO); options.setWriteDisposition(WriteDisposition.FAIL); - String targetRootPath = "gs://root/"; - String targetfilePathUnpartitionedTable = - String.format("%s/table1/output-table1.avro", targetRootPath); - String targetfilePathPartitionedTable = - String.format("%s/table1/p2_pid=partition1/output-table1-partition1.avro", targetRootPath); Filter f = - new DataplexBigQueryToGcsFilter( + new com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter( options, - targetRootPath, - new ArrayList() { - { - add(targetfilePathUnpartitionedTable); - add(targetfilePathPartitionedTable); - } - }); + Arrays.asList( + "table1/output-table1.avro", "table1/p2=partition1/output-table1-partition1.avro")); f.shouldSkipUnpartitionedTable(t); f.shouldSkipPartition(t, p); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/SchemasTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/SchemasTest.java index 63be4b1fda..160d63423d 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/SchemasTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/SchemasTest.java @@ -180,4 +180,25 @@ public void testJdbcSchemaToAvro() throws SQLException { assertEquals(expectedSchema, schema); } + + @Test + public void testRenameAvroField() { + Schema originalSchema = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"__root__\",\"fields\":" + + "[{\"name\":\"ts\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}]}," + + "{\"name\":\"___old___\",\"type\":[\"null\",\"string\"]}," + + "{\"name\":\"i1\",\"type\":[\"null\",\"long\"]}]}"); + + Schema actualSchema = Schemas.renameAvroField(originalSchema, "___old___", "___new___"); + + String expectedSchemaString = + "{\"type\":\"record\",\"name\":\"__root__\",\"fields\":" + + "[{\"name\":\"ts\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}]}," + + "{\"name\":\"___new___\",\"type\":[\"null\",\"string\"],\"aliases\":[\"___old___\"]}," + + "{\"name\":\"i1\",\"type\":[\"null\",\"long\"]}]}"; + + assertEquals(expectedSchemaString, actualSchema.toString()); + } } diff --git a/v2/googlecloud-to-googlecloud/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/v2/googlecloud-to-googlecloud/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..d99c786bc1 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,2 @@ +# Required to mock final methods: +mock-maker-inline \ No newline at end of file From 69971b41df6bbfbabd0cbd90f7ec8243b9ab7d26 Mon Sep 17 00:00:00 2001 From: anikin Date: Thu, 30 Dec 2021 19:07:02 -0800 Subject: [PATCH 012/145] Dataplex Tiering template: minor parameter changes. Renamed parameters: * tableRefs => tables * sourceBigQueryAssetName => sourceBigQueryDataset * destinationGcsBucketAssetName => destinateionStorageBucketAssetName sourceBigQueryDataset now supports both Dataplex asset IDs and BigQuery dataset IDs. PiperOrigin-RevId: 419033465 --- .../options/DataplexBigQueryToGcsOptions.java | 19 +++++++------ .../v2/templates/DataplexBigQueryToGcs.java | 19 +++++++------ .../v2/utils/DataplexBigQueryToGcsFilter.java | 4 +-- .../DataplexBigQueryToGcsFilterTest.java | 28 +++++++++---------- 4 files changed, 37 insertions(+), 33 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java index 466e1a23ac..b9968edc78 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java @@ -37,30 +37,31 @@ public interface DataplexBigQueryToGcsOptions UpdateDataplexBigQueryToGcsExportMetadataTransform.Options { @Description( - "Dataplex asset name for the the BigQuery dataset to tier data from. Format:" + "BigQuery dataset to tier data from. Format: " + " projects//locations//lakes//zones//assets/.") + + "-name> (Dataplex asset name) or projects//datasets/ (BigQuery" + + " dataset ID).") @Required - String getSourceBigQueryAssetName(); + String getSourceBigQueryDataset(); - void setSourceBigQueryAssetName(String sourceBigQueryAssetName); + void setSourceBigQueryDataset(String sourceBigQueryDataset); @Description( "A comma-separated list of BigQuery tables to tier. If none specified, all tables will be" + " tiered. Tables should be specified by their name only (no project/dataset prefix)." + " Case-sensitive!") - String getTableRefs(); + String getTables(); - void setTableRefs(String tableRefs); + void setTables(String tables); @Description( - "Dataplex asset name for the the GCS bucket to tier data to. Format:" + "Dataplex asset name for the the Cloud Storage bucket to tier data to. Format:" + " projects//locations//lakes//zones//assets/.") @Required - String getDestinationGcsBucketAssetName(); + String getDestinationStorageBucketAssetName(); - void setDestinationGcsBucketAssetName(String destinationGcsBucketAssetName); + void setDestinationStorageBucketAssetName(String destinationStorageBucketAssetName); @Description( "The parameter can either be: 1) unspecified, 2) date (and optional time) 3) Duration.\n" diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index 5b73ca4936..f98bf880d2 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -133,16 +133,19 @@ private static Pipeline setUpPipeline( String gcsResource = resolveAsset( dataplex, - options.getDestinationGcsBucketAssetName(), + options.getDestinationStorageBucketAssetName(), DataplexAssetResourceSpec.STORAGE_BUCKET); - - String bqResource = - resolveAsset( - dataplex, - options.getSourceBigQueryAssetName(), - DataplexAssetResourceSpec.BIGQUERY_DATASET); - String targetRootPath = "gs://" + gcsResource; + + String bqResource = options.getSourceBigQueryDataset(); + // This can be either a BigQuery dataset ID or a Dataplex Asset Name pointing to the dataset. + // Possible formats: + // projects//datasets/ + // projects//locations//lakes//zones//assets/ + // If param contains "/lakes/", assume it's a Dataplex resource and resolve it into BQ ID first: + if (bqResource.toLowerCase().contains("/lakes/")) { + bqResource = resolveAsset(dataplex, bqResource, DataplexAssetResourceSpec.BIGQUERY_DATASET); + } DatasetId datasetId = BigQueryUtils.parseDatasetUrn(bqResource); BigQueryMetadataLoader metadataLoader = diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java index ea7aeba14e..ed1ad87c79 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java @@ -58,12 +58,12 @@ public DataplexBigQueryToGcsFilter( this.maxLastModifiedTime = null; } - String tableRefs = options.getTableRefs(); + String tableRefs = options.getTables(); if (tableRefs != null && !tableRefs.isEmpty()) { List tableRefList = SPLITTER.splitToList(tableRefs); checkArgument( !tableRefList.isEmpty(), - "Got an non-empty tableRefs param '%s', but couldn't parse it into a valid table list," + "Got an non-empty tables param '%s', but couldn't parse it into a valid table list," + " please check its format.", tableRefs); this.includeTables = new HashSet<>(tableRefList); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java index 003169887b..3e11e8d222 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java @@ -54,7 +54,7 @@ public void test_whenNoFilterOptions_filterAcceptsAllTablesAndPartitions() { BigQueryTable.Builder t = table(); BigQueryTablePartition p = partition().build(); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -65,13 +65,13 @@ public void test_whenNoFilterOptions_filterAcceptsAllTablesAndPartitions() { } @Test - public void test_whenTableRefsSet_filterExcludesTablesByName() { + public void test_whenTablesSet_filterExcludesTablesByName() { BigQueryTable.Builder includedTable1 = table().setTableName("includedTable1"); BigQueryTable.Builder includedTable2 = table().setTableName("includedTable2"); BigQueryTable.Builder excludedTable = table().setTableName("excludedTable"); BigQueryTablePartition p = partition().build(); - options.setTableRefs("includedTable1,includedTable2"); + options.setTables("includedTable1,includedTable2"); options.setExportDataModifiedBeforeDateTime(null); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -91,8 +91,8 @@ public void test_whenTableRefsSet_filterExcludesTablesByName() { } @Test(expected = IllegalArgumentException.class) - public void test_whenTableRefsIsInvalid_throwsException() { - options.setTableRefs(","); + public void test_whenTablesIsInvalid_throwsException() { + options.setTables(","); new DataplexBigQueryToGcsFilter(options, new ArrayList()); } @@ -114,7 +114,7 @@ public void test_whenBeforeDateSet_filterExcludesTablesAndPartitions() { .build(); List partitions = Arrays.asList(olderPartition, newerPartition); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -137,7 +137,7 @@ public void test_whenBeforeDateHasTimeZone_timeParsedCorrectly() { BigQueryTable.Builder olderTable = table().setLastModificationTime(TS_MICROS_2021_01_01_15_00_00_UTC + 1000L); - options.setTableRefs(null); + options.setTables(null); { options.setExportDataModifiedBeforeDateTime("2021-01-01T15:00:00Z"); @@ -180,7 +180,7 @@ public void test_whenBeforeDateHasNoTime_dateParsedCorrectly() { BigQueryTable.Builder newerTable = table().setLastModificationTime(micros - 1000L); BigQueryTable.Builder olderTable = table().setLastModificationTime(micros + 1000L); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime("2021-02-15"); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -196,7 +196,7 @@ public void test_whenBeforeDateIs1DayDuration_dateParsedCorrectly() { BigQueryTable.Builder olderTable = table().setLastModificationTime(micros - 100000L); BigQueryTable.Builder newerTable = table().setLastModificationTime(micros + 100000L); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime("-P1D"); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -212,7 +212,7 @@ public void test_whenBeforeDateIs1Day3HoursDuration_dateParsedCorrectly() { BigQueryTable.Builder olderTable = table().setLastModificationTime(micros - 100000L); BigQueryTable.Builder newerTable = table().setLastModificationTime(micros + 100000L); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime("-p1dt3h"); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -222,7 +222,7 @@ public void test_whenBeforeDateIs1Day3HoursDuration_dateParsedCorrectly() { @Test public void test_whenPartitionedTableHasNoPartitions_filterExcludesTable() { - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); Filter f = new DataplexBigQueryToGcsFilter(options, new ArrayList()); @@ -235,7 +235,7 @@ public void test_whenTargetFileExistsWithWriteDispositionSKIP_filterExcludesTabl BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); BigQueryTablePartition p = partition().setPartitionName("partition1").build(); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); options.setFileFormat(FileFormat.AVRO); options.setWriteDisposition(WriteDisposition.SKIP); @@ -255,7 +255,7 @@ public void test_whenTargetFileExistsWithWriteDispositionOverwrite_filterAccepts BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); BigQueryTablePartition p = partition().setPartitionName("partition1").build(); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); options.setFileFormat(FileFormat.AVRO); options.setWriteDisposition(WriteDisposition.OVERWRITE); @@ -275,7 +275,7 @@ public void test_whenTargetFileExistsWithWriteDispositionFail_filterAcceptsTable BigQueryTable.Builder t = table().setTableName("table1").setPartitioningColumn("p2"); BigQueryTablePartition p = partition().setPartitionName("partition1").build(); - options.setTableRefs(null); + options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); options.setFileFormat(FileFormat.AVRO); options.setWriteDisposition(WriteDisposition.FAIL); From 04c2125751059326204dadfd707973d128517324 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Tue, 4 Jan 2022 10:19:55 -0800 Subject: [PATCH 013/145] Add parameter to allow users to override jdk.tls.disabledAlgorithms in JdbcToBigquery template. PiperOrigin-RevId: 419626058 --- pom.xml | 13 +++++-- .../teleport/templates/JdbcToBigQuery.java | 36 +++++++++++++++++++ .../templates/common/JdbcConverters.java | 12 ++++++- 3 files changed, 57 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index d1df694d5c..c7e9ae42a0 100644 --- a/pom.xml +++ b/pom.xml @@ -37,6 +37,7 @@ UTF-8 1.39.2 + 1.0-rc6 1.8.2 8.7 2.34.0 @@ -200,6 +201,11 @@ + + com.google.auto.service + auto-service + ${autovalue.service.version} + @@ -651,11 +657,12 @@ org.threeten:threetenbp org.json:json - org.apache.hadoop:hadoop-mapreduce-client-core + org.apache.hadoop:hadoop-mapreduce-client-core org.apache.hadoop:hadoop-common org.codehaus.jackson:jackson-core-asl - org.codehaus.jackson:jackson-mapper-asl - io.opencensus:opencensus-api + org.codehaus.jackson:jackson-mapper-asl + io.opencensus:opencensus-api + com.google.auto.service:auto-service diff --git a/src/main/java/com/google/cloud/teleport/templates/JdbcToBigQuery.java b/src/main/java/com/google/cloud/teleport/templates/JdbcToBigQuery.java index 6cb53940e8..bcdb20dea4 100644 --- a/src/main/java/com/google/cloud/teleport/templates/JdbcToBigQuery.java +++ b/src/main/java/com/google/cloud/teleport/templates/JdbcToBigQuery.java @@ -16,21 +16,57 @@ package com.google.cloud.teleport.templates; import com.google.api.services.bigquery.model.TableRow; +import com.google.auto.service.AutoService; import com.google.cloud.teleport.io.DynamicJdbcIO; import com.google.cloud.teleport.templates.common.JdbcConverters; import com.google.cloud.teleport.util.KMSEncryptedNestedValueProvider; +import java.security.Security; +import javax.net.ssl.SSLServerSocketFactory; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.harness.JvmInitializer; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A template that copies data from a relational database using JDBC to an existing BigQuery table. */ public class JdbcToBigQuery { + private static final Logger LOG = LoggerFactory.getLogger(JdbcToBigQuery.class); + + /** + * Custom JvmInitializer to override jdk.tls.disabledAlgorithms through the template parameters. + */ + @AutoService(JvmInitializer.class) + public static class CustomJvmInitializer implements JvmInitializer { + @Override + public void onStartup() {} + + @Override + public void beforeProcessing(PipelineOptions options) { + JdbcConverters.JdbcToBigQueryOptions pipelineOptions = + options.as(JdbcConverters.JdbcToBigQueryOptions.class); + if (pipelineOptions.getDisabledAlgorithms() != null + && pipelineOptions.getDisabledAlgorithms().get() != null) { + String value = pipelineOptions.getDisabledAlgorithms().get(); + // if the user sets disabledAlgorithms to "none" then set "jdk.tls.disabledAlgorithms" to "" + if (value.equals("none")) { + value = ""; + } + LOG.info("disabledAlgorithms is set to {}.", value); + Security.setProperty("jdk.tls.disabledAlgorithms", value); + SSLServerSocketFactory fact = (SSLServerSocketFactory) SSLServerSocketFactory.getDefault(); + LOG.info("Supported Ciper Suites: " + String.join("\n", fact.getSupportedCipherSuites())); + } + } + } + private static ValueProvider maybeDecrypt( ValueProvider unencryptedValue, ValueProvider kmsKey) { return new KMSEncryptedNestedValueProvider(unencryptedValue, kmsKey); diff --git a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java index 36f91166cc..938b5b1f32 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java @@ -85,10 +85,20 @@ public interface JdbcToBigQueryOptions extends PipelineOptions { void setBigQueryLoadingTemporaryDirectory(ValueProvider directory); @Description( - "KMS Encryption Key should be in the format projects/{gcp_project}/locations/{key_region}/keyRings/{key_ring}/cryptoKeys/{kms_key_name}") + "KMS Encryption Key should be in the format" + + " projects/{gcp_project}/locations/{key_region}/keyRings/{key_ring}/cryptoKeys/{kms_key_name}") ValueProvider getKMSEncryptionKey(); void setKMSEncryptionKey(ValueProvider keyName); + + @Description( + "Comma seperated algorithms to disable. If this value is set to \"none\" then" + + " jdk.tls.disabledAlgorithms is set to \"\". Use with care, as the algorithms" + + " disabled by default are known to have either vulnerabilities or performance issues." + + " for example: SSLv3, RC4.") + ValueProvider getDisabledAlgorithms(); + + void setDisabledAlgorithms(ValueProvider disabledAlgorithms); } /** Factory method for {@link ResultSetToTableRow}. */ From 9a14bd175adc475c37698f8a6b85b8c2da5a6fba Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 5 Jan 2022 09:16:58 -0800 Subject: [PATCH 014/145] Fix spotless issues. We're planning to start checking spotless on all PRs. This will minimize the problems that will create. PiperOrigin-RevId: 419844695 --- CONTRIBUTING.md | 6 +- .../teleport/spanner/ImportPipeline.java | 2 +- .../spanner/SpannerRecordConverter.java | 3 +- .../teleport/splunk/HttpEventPublisher.java | 5 +- .../templates/common/DatastoreConverters.java | 25 +- .../google/cloud/teleport/util/GCSUtils.java | 1 - .../sdk/io/gcp/spanner/LocalSpannerIO.java | 6 +- .../spanner/ExportRelatedTablesCheckTest.java | 304 ++++++------------ .../splunk/CustomX509TrustManagerTest.java | 15 +- .../splunk/HttpEventPublisherTest.java | 10 +- v2/bigquery-to-bigtable/README.md | 4 +- .../v2/templates/BigQueryToBigtable.java | 7 +- .../v2/transforms/BigQueryConverters.java | 8 +- .../v2/transforms/BigQueryConvertersTest.java | 97 ++---- .../FormatDatastreamRecordToJsonTest.java | 92 +++--- .../teleport/v2/transforms/CreateDml.java | 19 +- .../transforms/WriteToElasticsearch.java | 140 ++++---- .../utils/ElasticsearchUtils.java | 26 +- .../templates/PubSubToElasticsearch.java | 9 +- .../transforms/EventMetadataBuilder.java | 284 ++++++++-------- .../FailedPubsubMessageToPubsubTopicFn.java | 80 ++--- .../transforms/EventMetadataBuilderTest.java | 134 ++++---- .../cloud/teleport/v2/io/DynamicJdbcIO.java | 2 +- .../v2/templates/DataplexBigQueryToGcs.java | 3 +- .../beam/sdk/io/jdbc/BeamSchemaUtil.java | 3 +- 25 files changed, 570 insertions(+), 715 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index f7e72af8e7..42d31822a1 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -35,7 +35,7 @@ information on using pull requests. ## IntelliJ & IDE Setup For v1 templates, just open the project at the root directory and use maven -from there. +from there. -For v2 templates, open the project at the root directory then find the -"Add Maven Project" action and add the v2 directory and let the project rebuild. \ No newline at end of file +For v2 templates, open the project at the root directory then find the +"Add Maven Project" action and add the v2 directory and let the project rebuild. diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java index b13d072802..2b539fad00 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java @@ -98,7 +98,7 @@ public interface Options extends PipelineOptions { ValueProvider getDDLCreationTimeoutInMinutes(); void setDDLCreationTimeoutInMinutes(ValueProvider value); - + @Description("The spanner priority. --spannerPriority must be one of:[HIGH,MEDIUM,LOW]") ValueProvider getSpannerPriority(); diff --git a/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java b/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java index 0c88337e49..f7e2eed663 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java @@ -99,8 +99,7 @@ public GenericRecord convert(Struct row) { field, nullValue ? null : ByteBuffer.wrap(row.getBytes(fieldName).toByteArray())); break; case STRING: - if (STRING_PATTERN.matcher(spannerType).matches() - || spannerType.equals("JSON")) { + if (STRING_PATTERN.matcher(spannerType).matches() || spannerType.equals("JSON")) { builder.set(field, nullValue ? null : row.getString(fieldName)); } else if (spannerType.equals("TIMESTAMP")) { builder.set(field, nullValue ? null : row.getTimestamp(fieldName).toString()); diff --git a/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java b/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java index 55df9c6a93..2bc002809c 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java +++ b/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java @@ -356,7 +356,7 @@ private CloseableHttpClient getHttpClient( ? NoopHostnameVerifier.INSTANCE : new DefaultHostnameVerifier(); - SSLContext sslContext = SSLContextBuilder.create().build(); + SSLContext sslContext = SSLContextBuilder.create().build(); if (disableCertificateValidation) { LOG.info("Certificate validation is disabled"); sslContext = @@ -368,8 +368,7 @@ private CloseableHttpClient getHttpClient( InputStream inStream = new ByteArrayInputStream(rootCaCertificate); CertificateFactory cf = CertificateFactory.getInstance("X.509"); X509Certificate cert = (X509Certificate) cf.generateCertificate(inStream); - CustomX509TrustManager customTrustManager = - new CustomX509TrustManager(cert); + CustomX509TrustManager customTrustManager = new CustomX509TrustManager(cert); sslContext.init(null, new TrustManager[] {customTrustManager}, null); } diff --git a/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java index 71d0c3601d..8df19100e5 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java @@ -267,6 +267,7 @@ public abstract static class WriteJsonEntities public abstract ValueProvider hintNumWorkers(); public abstract Boolean throttleRampup(); + public abstract TupleTag errorTag(); /** Builder for WriteJsonEntities. */ @@ -277,6 +278,7 @@ public abstract static class Builder { public abstract Builder setHintNumWorkers(ValueProvider hintNumWorkers); public abstract Builder setThrottleRampup(Boolean throttleRampup); + public abstract Builder setErrorTag(TupleTag errorTag); public abstract WriteJsonEntities build(); @@ -291,9 +293,8 @@ public static Builder newBuilder() { @Override public PCollectionTuple expand(PCollection entityJson) { TupleTag goodTag = new TupleTag<>(); - DatastoreV1.Write datastoreWrite = DatastoreIO.v1().write() - .withProjectId(projectId()) - .withHintNumWorkers(hintNumWorkers()); + DatastoreV1.Write datastoreWrite = + DatastoreIO.v1().write().withProjectId(projectId()).withHintNumWorkers(hintNumWorkers()); if (!throttleRampup()) { datastoreWrite = datastoreWrite.withRampupThrottlingDisabled(); } @@ -305,9 +306,7 @@ public PCollectionTuple expand(PCollection entityJson) { "CheckSameKey", CheckSameKey.newBuilder().setErrorTag(errorTag()).setGoodTag(goodTag).build()); - entities - .get(goodTag) - .apply("WriteToDatastore", datastoreWrite); + entities.get(goodTag).apply("WriteToDatastore", datastoreWrite); return entities; } } @@ -412,6 +411,7 @@ public abstract static class Builder { public abstract Builder setHintNumWorkers(ValueProvider hintNumWorkers); public abstract Builder setThrottleRampup(Boolean throttleRampup); + public abstract DatastoreDeleteEntityJson build(); } @@ -423,7 +423,9 @@ public static Builder newBuilder() { @Override public PDone expand(PCollection entityJson) { - DatastoreV1.DeleteKey datastoreDelete = DatastoreIO.v1().deleteKey() + DatastoreV1.DeleteKey datastoreDelete = + DatastoreIO.v1() + .deleteKey() .withProjectId(projectId()) .withHintNumWorkers(hintNumWorkers()); if (!throttleRampup()) { @@ -718,9 +720,8 @@ public static Builder newBuilder() { @Override public PCollectionTuple expand(PCollection entity) { TupleTag goodTag = new TupleTag<>(); - DatastoreV1.Write datastoreWrite = DatastoreIO.v1().write() - .withProjectId(projectId()) - .withHintNumWorkers(hintNumWorkers()); + DatastoreV1.Write datastoreWrite = + DatastoreIO.v1().write().withProjectId(projectId()).withHintNumWorkers(hintNumWorkers()); if (!throttleRampup()) { datastoreWrite = datastoreWrite.withRampupThrottlingDisabled(); } @@ -734,9 +735,7 @@ public PCollectionTuple expand(PCollection entity) { entity.apply( "CheckSameKey", CheckSameKey.newBuilder().setErrorTag(errorTag()).setGoodTag(goodTag).build()); - entities - .get(goodTag) - .apply("WriteToDatastore", datastoreWrite); + entities.get(goodTag).apply("WriteToDatastore", datastoreWrite); return entities; } } diff --git a/src/main/java/com/google/cloud/teleport/util/GCSUtils.java b/src/main/java/com/google/cloud/teleport/util/GCSUtils.java index 27627f2f48..7bbcff469c 100644 --- a/src/main/java/com/google/cloud/teleport/util/GCSUtils.java +++ b/src/main/java/com/google/cloud/teleport/util/GCSUtils.java @@ -13,7 +13,6 @@ * License for the specific language governing permissions and limitations under * the License. */ - package com.google.cloud.teleport.util; import static java.util.stream.Collectors.toList; diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java index b550027c15..9c0e2c0539 100644 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java +++ b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java @@ -1487,9 +1487,9 @@ private void spannerWriteWithRetryIfSchemaChange(Iterable batch) if (spannerConfig.getRpcPriority() != null && spannerConfig.getRpcPriority().get() != null) { spannerAccessor - .getDatabaseClient() - .writeAtLeastOnceWithOptions( - batch, Options.priority(spannerConfig.getRpcPriority().get())); + .getDatabaseClient() + .writeAtLeastOnceWithOptions( + batch, Options.priority(spannerConfig.getRpcPriority().get())); } else { spannerAccessor.getDatabaseClient().writeAtLeastOnce(batch); } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java index d8e0f05149..e7f3cd10e4 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java @@ -124,6 +124,7 @@ private void createAndPopulate(Ddl ddl, int numBatches) throws Exception { /* Validates behavior of exporting full db without selecting any tables */ @Test public void exportWithoutTableSelection() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("Users") .column("first_name").string().max().endColumn() @@ -158,6 +159,7 @@ public void exportWithoutTableSelection() throws Exception { .primaryKey().asc("first_name").desc("last_name").asc("id").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, 100); @@ -176,7 +178,6 @@ public void exportWithoutTableSelection() throws Exception { compareExpectedTables( destDbPrefix + fullExportChkpt, ImmutableList.of(allTypesTable, peopleTable, usersTable)); - // Check to see selected tables exported with data and and unselected tables did not List exportTables = ImmutableList.of(allTypesTable, peopleTable, usersTable); List unselectedTables = Collections.emptyList(); @@ -186,6 +187,7 @@ public void exportWithoutTableSelection() throws Exception { /* Validates behavior of single table database exporting */ @Test public void exportSingleTable() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("Users") .column("first_name").string().max().endColumn() @@ -214,6 +216,7 @@ public void exportSingleTable() throws Exception { .primaryKey().asc("first_name").desc("last_name").asc("id").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, 100); // Export and import the table 'Users' from the database only @@ -239,6 +242,7 @@ public void exportSingleTable() throws Exception { /* Validates behavior of exporting multiple unrelated tables */ @Test public void exportMultipleTables() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("Users") .column("first_name").string().max().endColumn() @@ -273,6 +277,7 @@ public void exportMultipleTables() throws Exception { .primaryKey().asc("first_name").desc("last_name").asc("id").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, 100); @@ -300,6 +305,7 @@ public void exportMultipleTables() throws Exception { /* Validates behavior of exporting a single, empty table from a database */ @Test public void exportSingleEmptyTable() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("Users") .column("first_name").string().max().endColumn() @@ -328,18 +334,21 @@ public void exportSingleEmptyTable() throws Exception { .primaryKey().asc("first_name").desc("last_name").asc("id").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, 100); // Add empty table. + // spotless:off Ddl ddlEmptyTable = Ddl.builder() .createTable("empty_table") .column("first").string().max().endColumn() .column("second").string().size(5).endColumn() .column("value").int64().endColumn() .primaryKey().asc("first").desc("second").end() - .endTable() + .endTable() .build(); + // spotless:on spannerServer.updateDatabase(sourceDb, ddlEmptyTable.createTableStatements()); // Export an empty table from a database @@ -367,54 +376,29 @@ public void exportSingleEmptyTable() throws Exception { * --shouldExportRelatedTables paramters are not filled */ @Test public void exportDbWithoutTableNamesAndFlag_exportsFullDb() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .primaryKey().asc("id1").asc("id2").end() .endTable() .createTable("table_b") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .createTable("table_c") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() - .interleaveInParent("table_b") + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() + .interleaveInParent("table_b") .endTable() .build(); + // spotless:on // Add to referencedTable field (i.e. `table_c` would have a foreign key constraint // referencing `table_a` ) @@ -447,54 +431,29 @@ public void exportDbWithoutTableNamesAndFlag_exportsFullDb() throws Exception { @Test public void exportTableWithoutRelatedTablesAndWithoutFlag_exportsSelectedTable() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .primaryKey().asc("id1").asc("id2").end() .endTable() .createTable("table_b") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() - .interleaveInParent("table_a") + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() + .interleaveInParent("table_a") .endTable() .createTable("table_c") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, /* numBatches = */ 100); @@ -523,6 +482,7 @@ public void exportTableWithoutRelatedTablesAndWithoutFlag_exportsSelectedTable() * need to be exported */ @Test public void exportTableWithRelatedTablesAndWithoutFlag_stopsPipelineExecution() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") .column("id1").int64().endColumn() @@ -608,6 +568,7 @@ public void exportTableWithRelatedTablesAndWithoutFlag_stopsPipelineExecution() .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .build(); + // spotless:apply createAndPopulate(ddl, /* numBatches = */ 100); @@ -641,54 +602,29 @@ public void exportTableWithRelatedTablesAndWithoutFlag_stopsPipelineExecution() * --shouldExportRelatedTables is set to true */ @Test public void exportFullDbWithFlagTrue() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .primaryKey().asc("id1").asc("id2").end() .endTable() .createTable("table_b") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .createTable("table_c") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() - .interleaveInParent("table_b") + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() + .interleaveInParent("table_b") .endTable() .build(); + // spotless:on // Add to referencedTable field (i.e. `table_c` would have a foreign key constraint // referencing `table_a` ) @@ -716,54 +652,29 @@ public void exportFullDbWithFlagTrue() throws Exception { * --shouldExportRelatedTables is set to false (either intentionally or by default) */ @Test public void exportFullDbWithFlagFalse() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .primaryKey().asc("id1").asc("id2").end() .endTable() .createTable("table_b") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .createTable("table_c") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() - .interleaveInParent("table_b") + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() + .interleaveInParent("table_b") .endTable() .build(); + // spotless:on // Add to referencedTable field (i.e. `table_c` would have a foreign key constraint // referencing `table_a` ) @@ -795,74 +706,39 @@ public void exportFullDbWithFlagFalse() throws Exception { * --shouldExportRelatedTables is set to true, and additional tables need to be exported */ @Test public void exportSelectedAndNecessaryTables() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .primaryKey().asc("id1").asc("id2").end() .endTable() .createTable("table_b") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .createTable("table_c") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() - .interleaveInParent("table_b") - .foreignKeys( + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() + .interleaveInParent("table_b") + .foreignKeys( ImmutableList.of( "ALTER TABLE `table_c` ADD CONSTRAINT `fk1` FOREIGN KEY (`id1`) REFERENCES" + " `table_b` (`id1`)")) .endTable() .createTable("table_d") - .column("id1") - .int64() - .endColumn() - .column("id2") - .int64() - .endColumn() - .column("id3") - .int64() - .endColumn() - .primaryKey() - .asc("id1") - .asc("id2") - .asc("id3") - .end() + .column("id1").int64().endColumn() + .column("id2").int64().endColumn() + .column("id3").int64().endColumn() + .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, /* numBatches = */ 100); @@ -948,6 +824,7 @@ public void randomExportTest() throws Exception { * exist in the database */ @Test public void exportNonExistentTable_stopsPipelineExecution() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") .column("id1").int64().endColumn() @@ -968,6 +845,7 @@ public void exportNonExistentTable_stopsPipelineExecution() throws Exception { .interleaveInParent("table_b") .endTable() .build(); + // spotless:on // Add to referencedTable field (i.e. `table_c` would have a foreign key constraint // referencing `table_a`) @@ -996,6 +874,7 @@ public void exportNonExistentTable_stopsPipelineExecution() throws Exception { * to be exported. */ @Test public void exportSelectedAndNecessaryTablesInComplexDdl() throws Exception { + // spotless:off Ddl ddl = Ddl.builder() .createTable("table_a") .column("id1").int64().endColumn() @@ -1081,6 +960,7 @@ public void exportSelectedAndNecessaryTablesInComplexDdl() throws Exception { .primaryKey().asc("id1").asc("id2").asc("id3").end() .endTable() .build(); + // spotless:on createAndPopulate(ddl, /* numBatches = */ 100); diff --git a/src/test/java/com/google/cloud/teleport/splunk/CustomX509TrustManagerTest.java b/src/test/java/com/google/cloud/teleport/splunk/CustomX509TrustManagerTest.java index 11982729d1..51ac7ecf2d 100644 --- a/src/test/java/com/google/cloud/teleport/splunk/CustomX509TrustManagerTest.java +++ b/src/test/java/com/google/cloud/teleport/splunk/CustomX509TrustManagerTest.java @@ -56,25 +56,24 @@ public void setUp() .getResource("PubsubToSplunkTestData/UnrecognizedSelfSignedCertificate.crt") .getFile()); rootCa = (X509Certificate) cf.generateCertificate(rootCaInputStream); - recognizedSelfSignedCertificate = (X509Certificate) - cf.generateCertificate(recognizedInputStream); - unrecognizedSelfSignedCertificate = (X509Certificate) - cf.generateCertificate(unrecognizedInputStream); + recognizedSelfSignedCertificate = + (X509Certificate) cf.generateCertificate(recognizedInputStream); + unrecognizedSelfSignedCertificate = + (X509Certificate) cf.generateCertificate(unrecognizedInputStream); customTrustManager = new CustomX509TrustManager(rootCa); } /** - * Tests whether a recognized (user provided) self-signed certificate - * is accepted by TrustManager. - * */ + * Tests whether a recognized (user provided) self-signed certificate is accepted by TrustManager. + */ @Test public void testCustomX509TrustManagerWithRecognizedCertificate() throws CertificateException { customTrustManager.checkServerTrusted( new X509Certificate[] {recognizedSelfSignedCertificate}, "RSA"); } - /** Tests whether a unrecognized self-signed certificate is rejected by TrustManager.*/ + /** Tests whether a unrecognized self-signed certificate is rejected by TrustManager. */ @Test(expected = Exception.class) public void testCustomX509TrustManagerWithUnrecognizedCertificate() throws CertificateException { customTrustManager.checkServerTrusted( diff --git a/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java b/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java index 040e3692e7..d9b82d528b 100644 --- a/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java +++ b/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java @@ -163,8 +163,9 @@ public void genericURLTest() } @Test - public void configureBackOffDefaultTest() throws NoSuchAlgorithmException, - KeyStoreException, KeyManagementException, IOException, CertificateException { + public void configureBackOffDefaultTest() + throws NoSuchAlgorithmException, KeyStoreException, KeyManagementException, IOException, + CertificateException { HttpEventPublisher publisherDefaultBackOff = HttpEventPublisher.newBuilder() @@ -179,8 +180,9 @@ public void configureBackOffDefaultTest() throws NoSuchAlgorithmException, } @Test - public void configureBackOffCustomTest() throws NoSuchAlgorithmException, - KeyStoreException, KeyManagementException, IOException, CertificateException { + public void configureBackOffCustomTest() + throws NoSuchAlgorithmException, KeyStoreException, KeyManagementException, IOException, + CertificateException { int timeoutInMillis = 600000; // 10 minutes HttpEventPublisher publisherWithBackOff = diff --git a/v2/bigquery-to-bigtable/README.md b/v2/bigquery-to-bigtable/README.md index cd05391657..43331a388b 100644 --- a/v2/bigquery-to-bigtable/README.md +++ b/v2/bigquery-to-bigtable/README.md @@ -149,7 +149,7 @@ gcloud beta dataflow flex-template run ${JOB_NAME} \ --parameters readIdColumn=${READ_ID_COLUMN},bigtableWriteProjectId=${BIGTABLE_WRITE_PROJECT_ID},bigtableWriteInstanceId=${BIGTABLE_WRITE_INSTANCE_ID},bigtableWriteTableId=${BIGTABLE_WRITE_TABLE_ID},bigtableWriteColumnFamily=${BIGTABLE_WRITE_COLUMN_FAMILY} ``` -Note: The `^~^` prefix on readQuery is used to make `~` a delimiter instead of +Note: The `^~^` prefix on readQuery is used to make `~` a delimiter instead of commas. This allows commas to be used in the query. Read more about [gcloud topic escaping](https://cloud.google.com/sdk/gcloud/reference/topic/escaping). #### Example query @@ -158,4 +158,4 @@ Here is an example query using a public dataset. It combines a few values into a ``` export READ_QUERY="SELECT CONCAT(SenderCompID,'#', OrderID) as rowkey, * FROM bigquery-public-data.cymbal_investments.trade_capture_report LIMIT 100" -``` \ No newline at end of file +``` diff --git a/v2/bigquery-to-bigtable/src/main/java/com/google/cloud/teleport/v2/templates/BigQueryToBigtable.java b/v2/bigquery-to-bigtable/src/main/java/com/google/cloud/teleport/v2/templates/BigQueryToBigtable.java index e0393eb58c..0f96eabe11 100644 --- a/v2/bigquery-to-bigtable/src/main/java/com/google/cloud/teleport/v2/templates/BigQueryToBigtable.java +++ b/v2/bigquery-to-bigtable/src/main/java/com/google/cloud/teleport/v2/templates/BigQueryToBigtable.java @@ -98,7 +98,8 @@ public static void main(String[] args) { Pipeline pipeline = Pipeline.create(options); - pipeline.apply( + pipeline + .apply( "AvroToMutation", BigQueryIO.read( AvroToMutation.newBuilder() @@ -109,9 +110,7 @@ public static void main(String[] args) { .withoutValidation() .withTemplateCompatibility() .usingStandardSql()) - .apply( - "WriteToTable", - CloudBigtableIO.writeToTable(bigtableTableConfig)); + .apply("WriteToTable", CloudBigtableIO.writeToTable(bigtableTableConfig)); pipeline.run(); } diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java index 38a7ace4c4..af6f322313 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java @@ -93,10 +93,10 @@ public class BigQueryConverters { private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory(); - /** Converts from the BigQuery Avro format into Bigtable mutation. */ @AutoValue - public abstract static class AvroToMutation implements SerializableFunction { + public abstract static class AvroToMutation + implements SerializableFunction { public abstract String columnFamily(); @@ -132,9 +132,7 @@ public Mutation apply(SchemaAndRecord record) { String columnValue = row.get(columnName).toString(); // TODO(billyjacobson): handle other types and column families put.addColumn( - Bytes.toBytes(columnFamily()), - Bytes.toBytes(columnName), - Bytes.toBytes(columnValue)); + Bytes.toBytes(columnFamily()), Bytes.toBytes(columnName), Bytes.toBytes(columnValue)); } return put; } diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java index 43da940dc1..36b5403b56 100644 --- a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java @@ -69,64 +69,42 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Unit tests for {@link BigQueryConverters}. - */ +/** Unit tests for {@link BigQueryConverters}. */ @RunWith(JUnit4.class) public class BigQueryConvertersTest { static final TableRow ROW = new TableRow().set("id", "007").set("state", "CA").set("price", 26.23); - /** - * The tag for the main output of the json transformation. - */ + /** The tag for the main output of the json transformation. */ static final TupleTag> TRANSFORM_OUT = - new TupleTag>() { - }; - /** - * The tag for the dead-letter output of the json to table row transform. - */ + new TupleTag>() {}; + /** The tag for the dead-letter output of the json to table row transform. */ static final TupleTag> TRANSFORM_DEADLETTER_OUT = - new TupleTag>() { - }; - /** - * The tag for the main output of the json transformation. - */ + new TupleTag>() {}; + /** The tag for the main output of the json transformation. */ static final TupleTag> UDF_OUT = - new TupleTag>() { - }; - /** - * The tag for the dead-letter output of the json to table row transform. - */ + new TupleTag>() {}; + /** The tag for the dead-letter output of the json to table row transform. */ static final TupleTag> UDF_TRANSFORM_DEADLETTER_OUT = - new TupleTag>() { - }; - /** - * String/String Coder for FailsafeElement. - */ + new TupleTag>() {}; + /** String/String Coder for FailsafeElement. */ static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = FailsafeElementCoder.of( NullableCoder.of(StringUtf8Coder.of()), NullableCoder.of(StringUtf8Coder.of())); - /** - * TableRow/String Coder for FailsafeElement. - */ + /** TableRow/String Coder for FailsafeElement. */ static final FailsafeElementCoder FAILSAFE_TABLE_ROW_ELEMENT_CODER = FailsafeElementCoder.of(TableRowJsonCoder.of(), NullableCoder.of(StringUtf8Coder.of())); // Define the TupleTag's here otherwise the anonymous class will force the test method to // be serialized. - private static final TupleTag TABLE_ROW_TAG = new TupleTag() { - }; + private static final TupleTag TABLE_ROW_TAG = new TupleTag() {}; private static final TupleTag> FAILSAFE_ELM_TAG = - new TupleTag>() { - }; + new TupleTag>() {}; private static final String jsonifiedTableRow = "{\"id\":\"007\",\"state\":\"CA\",\"price\":26.23}"; private static final String udfOutputRow = "{\"id\":\"007\",\"state\":\"CA\",\"price\":26.23,\"someProp\":\"someValue\"}"; - @Rule - public final transient TestPipeline pipeline = TestPipeline.create(); - @Rule - public ExpectedException expectedException = ExpectedException.none(); + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public ExpectedException expectedException = ExpectedException.none(); private ValueProvider entityKind = StaticValueProvider.of("TestEntity"); private ValueProvider uniqueNameColumn = StaticValueProvider.of("id"); private ValueProvider namespace = StaticValueProvider.of("bq-to-ds-test"); @@ -196,9 +174,7 @@ public class BigQueryConvertersTest { private String dateTimeFieldDesc = "Full publication date"; private String dateTimeFieldValue = "2013-08-19 23:28:20.000567"; - /** - * Tests the {@link BigQueryConverters.FailsafeJsonToTableRow} transform with good input. - */ + /** Tests the {@link BigQueryConverters.FailsafeJsonToTableRow} transform with good input. */ @Test @Category(NeedsRunner.class) public void testFailsafeJsonToTableRowValidInput() { @@ -291,9 +267,7 @@ public void testFailsafeJsonToTableRowInvalidJSON() { pipeline.run(); } - /** - * Generates an Avro record with a single field. - */ + /** Generates an Avro record with a single field. */ private Record generateSingleFieldAvroRecord( String name, String type, String description, Object value) { Schema avroSchema = @@ -309,23 +283,17 @@ private Record generateSingleFieldAvroRecord( return builder.build(); } - /** - * Generates a short string Avro field. - */ + /** Generates a short string Avro field. */ private String generateShortStringField() { return String.format(avroFieldTemplate, shortStringField, "string", shortStringFieldDesc); } - /** - * Generates a long string Avro field. - */ + /** Generates a long string Avro field. */ private String generateLongStringField() { return String.format(avroFieldTemplate, longStringField, "string", longStringFieldDesc); } - /** - * Generate a BigQuery TableSchema with nested fields. - */ + /** Generate a BigQuery TableSchema with nested fields. */ private TableFieldSchema generateNestedTableFieldSchema() { return new TableFieldSchema() .setName("address") @@ -336,9 +304,7 @@ private TableFieldSchema generateNestedTableFieldSchema() { new TableFieldSchema().setName("street_name").setType("STRING"))); } - /** - * Generates an Avro record with a record field type. - */ + /** Generates an Avro record with a record field type. */ static Record generateNestedAvroRecord() { String avroRecordFieldSchema = new StringBuilder() @@ -504,22 +470,21 @@ public void testSchemaUtils() { assertThat(fields.get(0).getType()).isEqualTo(LegacySQLTypeName.STRING); } - /** - * Tests that {@link BigQueryConverters.AvroToMutation} creates a Mutation. - */ + /** Tests that {@link BigQueryConverters.AvroToMutation} creates a Mutation. */ @Test public void testAvroToMutation() { // Arrange String rowkey = "rowkey"; String columnFamily = "CF"; - AvroToMutation avroToMutation = AvroToMutation.newBuilder().setColumnFamily(columnFamily) - .setRowkey(rowkey).build(); - - TableSchema bqSchema = new TableSchema() - .setFields( - Arrays.asList( - new TableFieldSchema().setName(rowkey).setType("STRING"), - new TableFieldSchema().setName(shortStringField).setType("STRING"))); + AvroToMutation avroToMutation = + AvroToMutation.newBuilder().setColumnFamily(columnFamily).setRowkey(rowkey).build(); + + TableSchema bqSchema = + new TableSchema() + .setFields( + Arrays.asList( + new TableFieldSchema().setName(rowkey).setType("STRING"), + new TableFieldSchema().setName(shortStringField).setType("STRING"))); Schema avroSchema = new Schema.Parser() diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java index a800303b32..8d8c8b7603 100644 --- a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java @@ -62,55 +62,55 @@ public class FormatDatastreamRecordToJsonTest { private static final String EXPECTED_NUMERIC_RECORD = "{\"id\":2,\"bitty\":0,\"booly\":0,\"tiny\":-1,\"small\":-1,\"medium\":-1," - + "\"inty\":-1,\"big\":-1,\"floater\":1.2,\"doubler\":1.3," - + "\"decimaler\":\"11.22\",\"tinyu\":255,\"smallu\":65535,\"mediumu\":16777215," - + "\"intyu\":4294967295,\"bigu\":\"0\"," - + "\"_metadata_stream\":\"projects/545418958905/locations/us-central1/streams/stream31\"," - + "\"_metadata_timestamp\":1628184913," - + "\"_metadata_read_timestamp\":1628184913," - + "\"_metadata_read_method\":\"mysql-cdc-binlog\"," - + "\"_metadata_source_type\":\"mysql\"," - + "\"_metadata_deleted\":false," - + "\"_metadata_table\":\"numbers\"," - + "\"_metadata_change_type\":\"INSERT\"," - + "\"_metadata_primary_keys\":[\"id\"]," - + "\"_metadata_schema\":\"user1\"," - + "\"_metadata_log_file\":\"mysql-bin.000025\"," - + "\"_metadata_log_position\":\"78443804\"," - + "\"_metadata_source\":{\"table\":\"numbers\",\"database\":\"user1\"," - + "\"primary_keys\":[\"id\"],\"log_file\":\"mysql-bin.000025\"," - + "\"log_position\":78443804,\"change_type\":\"INSERT\",\"is_deleted\":false}}"; + + "\"inty\":-1,\"big\":-1,\"floater\":1.2,\"doubler\":1.3," + + "\"decimaler\":\"11.22\",\"tinyu\":255,\"smallu\":65535,\"mediumu\":16777215," + + "\"intyu\":4294967295,\"bigu\":\"0\"," + + "\"_metadata_stream\":\"projects/545418958905/locations/us-central1/streams/stream31\"," + + "\"_metadata_timestamp\":1628184913," + + "\"_metadata_read_timestamp\":1628184913," + + "\"_metadata_read_method\":\"mysql-cdc-binlog\"," + + "\"_metadata_source_type\":\"mysql\"," + + "\"_metadata_deleted\":false," + + "\"_metadata_table\":\"numbers\"," + + "\"_metadata_change_type\":\"INSERT\"," + + "\"_metadata_primary_keys\":[\"id\"]," + + "\"_metadata_schema\":\"user1\"," + + "\"_metadata_log_file\":\"mysql-bin.000025\"," + + "\"_metadata_log_position\":\"78443804\"," + + "\"_metadata_source\":{\"table\":\"numbers\",\"database\":\"user1\"," + + "\"primary_keys\":[\"id\"],\"log_file\":\"mysql-bin.000025\"," + + "\"log_position\":78443804,\"change_type\":\"INSERT\",\"is_deleted\":false}}"; private static final String EXPECTED_MYSQL_PEOPLE = "{\"id\":1," - + "\"email\":\"dylan@email.com\"," - + "\"first_name\":\"Dylan\"," - + "\"last_name\":\"Person\"," - + "\"gender\":\"M\"," - + "\"birth_date\":\"2020-01-01T00:00:00Z\"," - + "\"created_at\":\"2020-02-12T00:00:00Z\"," - + "\"datetime_at\":\"2020-02-12T00:00:00Z\"," - + "\"_0col\":1," - + "\"_metadata_stream\":" - + "\"projects/269744978479/locations/us-central1/streams/datastream-test-fbefaf33\"," - + "\"_metadata_timestamp\":1623459160," - + "\"_metadata_read_timestamp\":1623459161," - + "\"_metadata_read_method\":\"mysql-backfill-fulldump\"," - + "\"_metadata_source_type\":\"mysql\"," - + "\"_metadata_deleted\":false,\"_metadata_table\":\"people\"," - + "\"_metadata_change_type\":\"INSERT\"," - + "\"_metadata_primary_keys\":[\"id\"]," - + "\"_metadata_schema\":\"test\"," - + "\"_metadata_log_file\":null," - + "\"_metadata_log_position\":null," - + "\"_metadata_source\":{" - + "\"table\":\"people\"," - + "\"database\":\"test\"," - + "\"primary_keys\":[\"id\"]," - + "\"log_file\":null," - + "\"log_position\":null," - + "\"change_type\":\"INSERT\"," - + "\"is_deleted\":false}}"; + + "\"email\":\"dylan@email.com\"," + + "\"first_name\":\"Dylan\"," + + "\"last_name\":\"Person\"," + + "\"gender\":\"M\"," + + "\"birth_date\":\"2020-01-01T00:00:00Z\"," + + "\"created_at\":\"2020-02-12T00:00:00Z\"," + + "\"datetime_at\":\"2020-02-12T00:00:00Z\"," + + "\"_0col\":1," + + "\"_metadata_stream\":" + + "\"projects/269744978479/locations/us-central1/streams/datastream-test-fbefaf33\"," + + "\"_metadata_timestamp\":1623459160," + + "\"_metadata_read_timestamp\":1623459161," + + "\"_metadata_read_method\":\"mysql-backfill-fulldump\"," + + "\"_metadata_source_type\":\"mysql\"," + + "\"_metadata_deleted\":false,\"_metadata_table\":\"people\"," + + "\"_metadata_change_type\":\"INSERT\"," + + "\"_metadata_primary_keys\":[\"id\"]," + + "\"_metadata_schema\":\"test\"," + + "\"_metadata_log_file\":null," + + "\"_metadata_log_position\":null," + + "\"_metadata_source\":{" + + "\"table\":\"people\"," + + "\"database\":\"test\"," + + "\"primary_keys\":[\"id\"]," + + "\"log_file\":null," + + "\"log_position\":null," + + "\"change_type\":\"INSERT\"," + + "\"is_deleted\":false}}"; @Test public void testParseAvroGenRecord() throws IOException, URISyntaxException { diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/CreateDml.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/CreateDml.java index bbbdbf0c7e..2e16a2d694 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/CreateDml.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/CreateDml.java @@ -35,8 +35,9 @@ * The {@code CreateDml} class batches data to ensure connection limits and builds the DmlInfo * objects. */ -public class CreateDml extends PTransform>, - PCollection>> { +public class CreateDml + extends PTransform< + PCollection>, PCollection>> { private static final Logger LOG = LoggerFactory.getLogger(CreateDml.class); private static final String WINDOW_DURATION = "1s"; @@ -48,8 +49,7 @@ private CreateDml(DataSourceConfiguration dataSourceConfiguration) { this.dataSourceConfiguration = dataSourceConfiguration; } - public static CreateDml of( - DataSourceConfiguration dataSourceConfiguration) { + public static CreateDml of(DataSourceConfiguration dataSourceConfiguration) { return new CreateDml(dataSourceConfiguration); } @@ -70,11 +70,10 @@ public DatastreamToDML getDatastreamToDML() { break; default: throw new IllegalArgumentException( - String.format("Database Driver %s is not supported.", driverName)); + String.format("Database Driver %s is not supported.", driverName)); } - return datastreamToDML - .withSchemaMap(schemaMap); + return datastreamToDML.withSchemaMap(schemaMap); } @Override @@ -82,9 +81,9 @@ public PCollection> expand( PCollection> input) { DatastreamToDML datastreamToDML = getDatastreamToDML(); return input - .apply("Reshuffle Into Buckets", - Reshuffle.>viaRandomKey() - .withNumBuckets(NUM_THREADS)) + .apply( + "Reshuffle Into Buckets", + Reshuffle.>viaRandomKey().withNumBuckets(NUM_THREADS)) .apply("Format to Postgres DML", ParDo.of(datastreamToDML)); } } diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/WriteToElasticsearch.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/WriteToElasticsearch.java index 7247c6c153..c88964be49 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/WriteToElasticsearch.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/WriteToElasticsearch.java @@ -51,92 +51,92 @@ @AutoValue public abstract class WriteToElasticsearch extends PTransform, PDone> { - /** Convert provided long to {@link Duration}. */ - private static Duration getDuration(Long milliseconds) { - return new Duration(milliseconds); + /** Convert provided long to {@link Duration}. */ + private static Duration getDuration(Long milliseconds) { + return new Duration(milliseconds); + } + + public static Builder newBuilder() { + return new AutoValue_WriteToElasticsearch.Builder(); + } + + public abstract ElasticsearchWriteOptions options(); + + /** + * Types have been removed in ES 7.0. Default will be _doc. See + * https://www.elastic.co/guide/en/elasticsearch/reference/current/removal-of-types.html" + */ + private static final String DOCUMENT_TYPE = "_doc"; + + @Override + public PDone expand(PCollection jsonStrings) { + ConnectionInformation connectionInformation = + new ConnectionInformation(options().getConnectionUrl()); + + ElasticsearchIO.ConnectionConfiguration config = + ElasticsearchIO.ConnectionConfiguration.create( + new String[] {connectionInformation.getElasticsearchURL().toString()}, + options().getIndex(), + DOCUMENT_TYPE); + + // If username and password are not blank, use them instead of ApiKey + if (StringUtils.isNotBlank(options().getElasticsearchUsername()) + && StringUtils.isNotBlank(options().getElasticsearchPassword())) { + config = + config + .withUsername(options().getElasticsearchUsername()) + .withPassword(options().getElasticsearchPassword()); + } else { + config = config.withApiKey(options().getApiKey()); } - public static Builder newBuilder() { - return new AutoValue_WriteToElasticsearch.Builder(); - } + ElasticsearchIO.Write elasticsearchWriter = + ElasticsearchIO.write() + .withConnectionConfiguration(config) + .withMaxBatchSize(options().getBatchSize()) + .withMaxBatchSizeBytes(options().getBatchSizeBytes()); - public abstract ElasticsearchWriteOptions options(); - - /** - * Types have been removed in ES 7.0. Default will be _doc. - * See https://www.elastic.co/guide/en/elasticsearch/reference/current/removal-of-types.html" - */ - private static final String DOCUMENT_TYPE="_doc"; - - @Override - public PDone expand(PCollection jsonStrings) { - ConnectionInformation connectionInformation = new ConnectionInformation(options().getConnectionUrl()); - - ElasticsearchIO.ConnectionConfiguration config = - ElasticsearchIO.ConnectionConfiguration.create( - new String[]{connectionInformation.getElasticsearchURL().toString()}, - options().getIndex(), - DOCUMENT_TYPE); - - //If username and password are not blank, use them instead of ApiKey - if (StringUtils.isNotBlank(options().getElasticsearchUsername()) - && StringUtils.isNotBlank(options().getElasticsearchPassword())) { - config = config - .withUsername(options().getElasticsearchUsername()) - .withPassword(options().getElasticsearchPassword()); - } else { - config = config.withApiKey(options().getApiKey()); - } - - ElasticsearchIO.Write elasticsearchWriter = - ElasticsearchIO.write() - .withConnectionConfiguration(config) - .withMaxBatchSize(options().getBatchSize()) - .withMaxBatchSizeBytes(options().getBatchSizeBytes()); - - if (Optional.ofNullable(options().getMaxRetryAttempts()).isPresent()) { - elasticsearchWriter.withRetryConfiguration( - ElasticsearchIO.RetryConfiguration.create( - options().getMaxRetryAttempts(), getDuration(options().getMaxRetryDuration()))); - } - - return jsonStrings.apply("WriteDocuments", elasticsearchWriter); + if (Optional.ofNullable(options().getMaxRetryAttempts()).isPresent()) { + elasticsearchWriter.withRetryConfiguration( + ElasticsearchIO.RetryConfiguration.create( + options().getMaxRetryAttempts(), getDuration(options().getMaxRetryDuration()))); } - /** Builder for {@link WriteToElasticsearch}. */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setOptions(ElasticsearchWriteOptions options); + return jsonStrings.apply("WriteDocuments", elasticsearchWriter); + } + + /** Builder for {@link WriteToElasticsearch}. */ + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setOptions(ElasticsearchWriteOptions options); - abstract ElasticsearchWriteOptions options(); + abstract ElasticsearchWriteOptions options(); - abstract WriteToElasticsearch autoBuild(); + abstract WriteToElasticsearch autoBuild(); - public WriteToElasticsearch build() { + public WriteToElasticsearch build() { - checkArgument( - options().getConnectionUrl() != null, "ConnectionUrl is required."); + checkArgument(options().getConnectionUrl() != null, "ConnectionUrl is required."); - checkArgument( - options().getApiKey() != null, "ApiKey is required."); + checkArgument(options().getApiKey() != null, "ApiKey is required."); - checkArgument(options().getIndex() != null, "Elasticsearch index should not be null."); + checkArgument(options().getIndex() != null, "Elasticsearch index should not be null."); checkArgument( options().getBatchSize() > 0, "Batch size must be > 0. Got: " + options().getBatchSize()); - checkArgument( - options().getBatchSizeBytes() > 0, - "Batch size bytes must be > 0. Got: " + options().getBatchSizeBytes()); + checkArgument( + options().getBatchSizeBytes() > 0, + "Batch size bytes must be > 0. Got: " + options().getBatchSizeBytes()); - /* Check that both {@link RetryConfiguration} parameters are supplied. */ - if (options().getMaxRetryAttempts() != null || options().getMaxRetryDuration() != null) { - checkArgument( - options().getMaxRetryDuration() != null && options().getMaxRetryAttempts() != null, - "Both max retry duration and max attempts must be supplied."); - } + /* Check that both {@link RetryConfiguration} parameters are supplied. */ + if (options().getMaxRetryAttempts() != null || options().getMaxRetryDuration() != null) { + checkArgument( + options().getMaxRetryDuration() != null && options().getMaxRetryAttempts() != null, + "Both max retry duration and max attempts must be supplied."); + } - return autoBuild(); - } + return autoBuild(); } + } } diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchUtils.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchUtils.java index 9c908e82cd..559f576a85 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchUtils.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchUtils.java @@ -18,21 +18,19 @@ import com.fasterxml.jackson.databind.JsonNode; import java.util.NoSuchElementException; -/** - * Miscellaneous util methods for googlecloud-to-elasticsearch. - */ +/** Miscellaneous util methods for googlecloud-to-elasticsearch. */ public class ElasticsearchUtils { - public static String getTimestampFromOriginalPayload(JsonNode node) throws NoSuchElementException { - if(node.has("timestamp")) { - return node.get("timestamp").asText(); - } else { - if (node.has("protoPayload") - && node.get("protoPayload").has("timestamp")) { - return node.get("protoPayload").get("timestamp").asText(); - } - } - - throw new NoSuchElementException("Unable to find \"timestamp\" value"); + public static String getTimestampFromOriginalPayload(JsonNode node) + throws NoSuchElementException { + if (node.has("timestamp")) { + return node.get("timestamp").asText(); + } else { + if (node.has("protoPayload") && node.get("protoPayload").has("timestamp")) { + return node.get("protoPayload").get("timestamp").asText(); + } } + + throw new NoSuchElementException("Unable to find \"timestamp\" value"); + } } diff --git a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/templates/PubSubToElasticsearch.java b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/templates/PubSubToElasticsearch.java index 6c58afae23..f8f1ba371c 100644 --- a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/templates/PubSubToElasticsearch.java +++ b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/templates/PubSubToElasticsearch.java @@ -158,12 +158,9 @@ public static PipelineResult run(PubSubToElasticsearchOptions options) { * Step 3b: Write elements that failed processing to error output PubSub topic via {@link PubSubIO}. */ convertedPubsubMessages - .get(TRANSFORM_ERROROUTPUT_OUT) - .apply(ParDo.of(new FailedPubsubMessageToPubsubTopicFn())) - .apply( - "writeFailureMessages", - PubsubIO.writeMessages().to(options.getErrorOutputTopic())); - + .get(TRANSFORM_ERROROUTPUT_OUT) + .apply(ParDo.of(new FailedPubsubMessageToPubsubTopicFn())) + .apply("writeFailureMessages", PubsubIO.writeMessages().to(options.getErrorOutputTopic())); // Execute the pipeline and return the result. return pipeline.run(); diff --git a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilder.java b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilder.java index 118d3ad939..dc51096850 100644 --- a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilder.java +++ b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilder.java @@ -27,168 +27,188 @@ import java.util.NoSuchElementException; /** - * EventMetadataBuilder is used to insert metadata required by Elasticsearch. - * The metadata helps Elasticsearch to visualize events on the dashboards, - * also uniform message format is needed for data analytics. + * EventMetadataBuilder is used to insert metadata required by Elasticsearch. The metadata helps + * Elasticsearch to visualize events on the dashboards, also uniform message format is needed for + * data analytics. + * *

Please refer to * inputGCPAuditlogMessageEnriched.json to see an example of enriched message. */ public class EventMetadataBuilder implements Serializable { + @JsonProperty("@timestamp") + private String timestamp; + + @JsonProperty("agent") + private Agent agent; + + @JsonProperty("data_stream") + private DataStream dataStream; + + @JsonProperty("ecs") + private Ecs ecs; + + @JsonProperty("message") + private String message; + + @JsonProperty("service") + private Service service; + + @JsonProperty("event") + private Event event; + + @JsonIgnore private String inputMessage; + @JsonIgnore private JsonNode enrichedMessage; + @JsonIgnore final ObjectMapper objectMapper = new ObjectMapper(); + @JsonIgnore EventMetadata eventMetadata; + + private EventMetadataBuilder( + String inputMessage, PubSubToElasticsearchOptions pubSubToElasticsearchOptions) { + eventMetadata = new EventMetadata(); + + try { + eventMetadata.timestamp = + ElasticsearchUtils.getTimestampFromOriginalPayload(objectMapper.readTree(inputMessage)); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Cannot parse input message as JSON: " + inputMessage, e); + } catch (NoSuchElementException e) { + // if timestamp is not found, we generate it + eventMetadata.timestamp = + new java.sql.Timestamp(System.currentTimeMillis()).toInstant().toString(); + } + + this.inputMessage = inputMessage; + + eventMetadata.ecs = new Ecs(); + eventMetadata.message = inputMessage; + eventMetadata.agent = new Agent(); + eventMetadata.agent.version = pubSubToElasticsearchOptions.getElasticsearchTemplateVersion(); + + eventMetadata.dataStream = new DataStream(); + eventMetadata.dataStream.dataset = pubSubToElasticsearchOptions.getDataset().getKeyWithPrefix(); + eventMetadata.dataStream.namespace = pubSubToElasticsearchOptions.getNamespace(); + + eventMetadata.service = new Service(); + eventMetadata.event = new Event(); + eventMetadata.service.type = pubSubToElasticsearchOptions.getDataset().getKeyWithPrefix(); + eventMetadata.event.dataset = pubSubToElasticsearchOptions.getDataset().getKeyWithPrefix(); + } + + public static EventMetadataBuilder build( + String inputMessage, PubSubToElasticsearchOptions pubSubToElasticsearchOptions) { + return new EventMetadataBuilder(inputMessage, pubSubToElasticsearchOptions); + } + + private void enrich() { + try { + enrichedMessage = objectMapper.readTree(inputMessage); + ((ObjectNode) enrichedMessage).putAll((ObjectNode) objectMapper.valueToTree(eventMetadata)); + ((ObjectNode) enrichedMessage).remove("timestamp"); + } catch (JsonProcessingException e) { + throw new IllegalStateException( + "Exception occurred while processing input message: " + inputMessage, e); + } + } + + public String getEnrichedMessageAsString() { + if (enrichedMessage == null) { + this.enrich(); + } + + try { + return objectMapper.writeValueAsString(enrichedMessage); + } catch (JsonProcessingException e) { + throw new IllegalStateException( + "Exception occurred while building enriched message: " + enrichedMessage, e); + } + } + + public JsonNode getEnrichedMessageAsJsonNode() { + if (enrichedMessage == null) { + this.enrich(); + } + + return enrichedMessage; + } + + @Override + public String toString() { + try { + return objectMapper.writeValueAsString(enrichedMessage); + } catch (JsonProcessingException e) { + throw new IllegalStateException( + "Exception occurred while writing EventMetadataBuilder as String.", e); + } + } + + static class EventMetadata { @JsonProperty("@timestamp") private String timestamp; + @JsonProperty("agent") private Agent agent; + @JsonProperty("data_stream") private DataStream dataStream; + @JsonProperty("ecs") private Ecs ecs; + @JsonProperty("message") private String message; + @JsonProperty("service") private Service service; + @JsonProperty("event") private Event event; - @JsonIgnore - private String inputMessage; - @JsonIgnore - private JsonNode enrichedMessage; - @JsonIgnore - final ObjectMapper objectMapper = new ObjectMapper(); - @JsonIgnore - EventMetadata eventMetadata; - - private EventMetadataBuilder(String inputMessage, PubSubToElasticsearchOptions pubSubToElasticsearchOptions) { - eventMetadata = new EventMetadata(); - - try { - eventMetadata.timestamp = ElasticsearchUtils.getTimestampFromOriginalPayload(objectMapper.readTree(inputMessage)); - } catch (JsonProcessingException e) { - throw new IllegalStateException("Cannot parse input message as JSON: " + inputMessage, e); - } catch (NoSuchElementException e) { - //if timestamp is not found, we generate it - eventMetadata.timestamp = new java.sql.Timestamp(System.currentTimeMillis()).toInstant().toString(); - } - - this.inputMessage = inputMessage; - - eventMetadata.ecs = new Ecs(); - eventMetadata.message = inputMessage; - eventMetadata.agent = new Agent(); - eventMetadata.agent.version = pubSubToElasticsearchOptions.getElasticsearchTemplateVersion(); - - eventMetadata.dataStream = new DataStream(); - eventMetadata.dataStream.dataset = pubSubToElasticsearchOptions.getDataset().getKeyWithPrefix(); - eventMetadata.dataStream.namespace = pubSubToElasticsearchOptions.getNamespace(); - - eventMetadata.service = new Service(); - eventMetadata.event = new Event(); - eventMetadata.service.type = pubSubToElasticsearchOptions.getDataset().getKeyWithPrefix(); - eventMetadata.event.dataset = pubSubToElasticsearchOptions.getDataset().getKeyWithPrefix(); - } - public static EventMetadataBuilder build(String inputMessage, PubSubToElasticsearchOptions pubSubToElasticsearchOptions) { - return new EventMetadataBuilder(inputMessage, pubSubToElasticsearchOptions); - } + @JsonIgnore private String inputMessage; + @JsonIgnore private JsonNode enrichedMessage; + @JsonIgnore final ObjectMapper objectMapper = new ObjectMapper(); + } - private void enrich() { - try { - enrichedMessage = objectMapper.readTree(inputMessage); - ((ObjectNode) enrichedMessage).putAll((ObjectNode) objectMapper.valueToTree(eventMetadata)); - ((ObjectNode) enrichedMessage).remove("timestamp"); - } catch (JsonProcessingException e) { - throw new IllegalStateException("Exception occurred while processing input message: " + inputMessage, e); - } - } + private static class Agent { + @JsonProperty("type") + private final String type = "dataflow"; - public String getEnrichedMessageAsString() { - if (enrichedMessage == null) { - this.enrich(); - } + @JsonProperty("name") + private final String name = ""; - try { - return objectMapper.writeValueAsString(enrichedMessage); - } catch (JsonProcessingException e) { - throw new IllegalStateException("Exception occurred while building enriched message: " + enrichedMessage, e); - } - } + @JsonProperty("version") + private String version; - public JsonNode getEnrichedMessageAsJsonNode() { - if (enrichedMessage == null) { - this.enrich(); - } + @JsonProperty("id") + private final String id = ""; + } - return enrichedMessage; - } - - @Override - public String toString() { - try { - return objectMapper.writeValueAsString(enrichedMessage); - } catch (JsonProcessingException e) { - throw new IllegalStateException("Exception occurred while writing EventMetadataBuilder as String.", e); - } - } - - static class EventMetadata { - @JsonProperty("@timestamp") - private String timestamp; - @JsonProperty("agent") - private Agent agent; - @JsonProperty("data_stream") - private DataStream dataStream; - @JsonProperty("ecs") - private Ecs ecs; - @JsonProperty("message") - private String message; - @JsonProperty("service") - private Service service; - @JsonProperty("event") - private Event event; - @JsonIgnore - private String inputMessage; - @JsonIgnore - private JsonNode enrichedMessage; - @JsonIgnore - final ObjectMapper objectMapper = new ObjectMapper(); - } + private static class DataStream { + @JsonProperty("type") + private final String type = "logs"; - private static class Agent { - @JsonProperty("type") - private final String type = "dataflow"; - @JsonProperty("name") - private final String name = ""; - @JsonProperty("version") - private String version; - @JsonProperty("id") - private final String id = ""; - } + @JsonProperty("dataset") + private String dataset; - private static class DataStream { - @JsonProperty("type") - private final String type = "logs"; - @JsonProperty("dataset") - private String dataset; - @JsonProperty("namespace") - private String namespace; - } + @JsonProperty("namespace") + private String namespace; + } - private static class Ecs { - @JsonProperty("version") - private final String version = "1.10.0"; - } + private static class Ecs { + @JsonProperty("version") + private final String version = "1.10.0"; + } - private static class Service { - @JsonProperty("type") - private String type; - } + private static class Service { + @JsonProperty("type") + private String type; + } - private static class Event { - @JsonProperty("module") - private final String module = "gcp"; - @JsonProperty("dataset") - private String dataset; - } + private static class Event { + @JsonProperty("module") + private final String module = "gcp"; + @JsonProperty("dataset") + private String dataset; + } } diff --git a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/FailedPubsubMessageToPubsubTopicFn.java b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/FailedPubsubMessageToPubsubTopicFn.java index c00afb6de9..821a878abb 100644 --- a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/FailedPubsubMessageToPubsubTopicFn.java +++ b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/FailedPubsubMessageToPubsubTopicFn.java @@ -28,53 +28,55 @@ import org.apache.beam.sdk.transforms.DoFn; /** - * The {@link FailedPubsubMessageToPubsubTopicFn} converts PubSub message which have failed processing into - * {@link com.google.api.services.pubsub.model.PubsubMessage} objects which can be output to a PubSub topic. + * The {@link FailedPubsubMessageToPubsubTopicFn} converts PubSub message which have failed + * processing into {@link com.google.api.services.pubsub.model.PubsubMessage} objects which can be + * output to a PubSub topic. */ public class FailedPubsubMessageToPubsubTopicFn - extends DoFn, PubsubMessage> { + extends DoFn, PubsubMessage> { - private static final String ERROR_MESSAGE = "errorMessage"; - private static final String ERROR_STACKTRACE = "stackTrace"; - private static final String ERROR_TIMESTAMP = "timestamp"; - private static final String ERROR_PAYLOAD_STRING = "payloadString"; - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String ERROR_MESSAGE = "errorMessage"; + private static final String ERROR_STACKTRACE = "stackTrace"; + private static final String ERROR_TIMESTAMP = "timestamp"; + private static final String ERROR_PAYLOAD_STRING = "payloadString"; + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - /** Counter to track total failed messages. */ - private static final Counter ERROR_MESSAGES_COUNTER = - Metrics.counter(FailedPubsubMessageToPubsubTopicFn.class, "total-failed-messages"); + /** Counter to track total failed messages. */ + private static final Counter ERROR_MESSAGES_COUNTER = + Metrics.counter(FailedPubsubMessageToPubsubTopicFn.class, "total-failed-messages"); - @ProcessElement - public void processElement(ProcessContext context) { - FailsafeElement failsafeElement = context.element(); - PubsubMessage pubsubMessage = failsafeElement.getOriginalPayload(); - String message = - pubsubMessage.getPayload().length > 0 - ? new String(pubsubMessage.getPayload()) - : pubsubMessage.getAttributeMap().toString(); + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement failsafeElement = context.element(); + PubsubMessage pubsubMessage = failsafeElement.getOriginalPayload(); + String message = + pubsubMessage.getPayload().length > 0 + ? new String(pubsubMessage.getPayload()) + : pubsubMessage.getAttributeMap().toString(); - // Format the timestamp for insertion - String timestamp; - try { - timestamp = ElasticsearchUtils.getTimestampFromOriginalPayload(OBJECT_MAPPER.readTree(message)); - } catch (Exception e) { - timestamp = new java.sql.Timestamp(System.currentTimeMillis()).toInstant().toString(); - } - - // Build the output PubSub message - ObjectNode outputMessage = OBJECT_MAPPER.createObjectNode(); - outputMessage - .put(ERROR_TIMESTAMP, timestamp) - .put(ERROR_MESSAGE, failsafeElement.getErrorMessage()) - .put(ERROR_STACKTRACE, failsafeElement.getStacktrace()) - .put(ERROR_PAYLOAD_STRING, message); + // Format the timestamp for insertion + String timestamp; + try { + timestamp = + ElasticsearchUtils.getTimestampFromOriginalPayload(OBJECT_MAPPER.readTree(message)); + } catch (Exception e) { + timestamp = new java.sql.Timestamp(System.currentTimeMillis()).toInstant().toString(); + } - ERROR_MESSAGES_COUNTER.inc(); + // Build the output PubSub message + ObjectNode outputMessage = OBJECT_MAPPER.createObjectNode(); + outputMessage + .put(ERROR_TIMESTAMP, timestamp) + .put(ERROR_MESSAGE, failsafeElement.getErrorMessage()) + .put(ERROR_STACKTRACE, failsafeElement.getStacktrace()) + .put(ERROR_PAYLOAD_STRING, message); - Map attributes = new HashMap<>(pubsubMessage.getAttributeMap()); - attributes.put(ERROR_MESSAGE, failsafeElement.getErrorMessage()); + ERROR_MESSAGES_COUNTER.inc(); - context.output(new PubsubMessage(outputMessage.toString().getBytes(StandardCharsets.UTF_8), attributes)); - } + Map attributes = new HashMap<>(pubsubMessage.getAttributeMap()); + attributes.put(ERROR_MESSAGE, failsafeElement.getErrorMessage()); + context.output( + new PubsubMessage(outputMessage.toString().getBytes(StandardCharsets.UTF_8), attributes)); + } } diff --git a/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilderTest.java b/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilderTest.java index 483b133bb8..2c59a99ae3 100644 --- a/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilderTest.java +++ b/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/transforms/EventMetadataBuilderTest.java @@ -32,40 +32,38 @@ import org.junit.rules.ExpectedException; /** - * Test cases for {@link EventMetadataBuilder}. - * Used to test the correctness of {@link EventMetadataBuilder} implementation, - * including insertion of metadata and error handling. + * Test cases for {@link EventMetadataBuilder}. Used to test the correctness of {@link + * EventMetadataBuilder} implementation, including insertion of metadata and error handling. */ public class EventMetadataBuilderTest { - private static final String RESOURCES_DIR = "EventMetadataBuilder/"; - private static final String INPUT_MESSAGE_FILE_PATH = - Resources.getResource(RESOURCES_DIR + "inputGCPAuditlogMessage.json").getPath(); - private static final String INPUT_MESSAGE_INVALID_FILE_PATH = - Resources.getResource(RESOURCES_DIR + "inputGCPAuditlogMessageInvalid.json").getPath(); - private static final boolean IS_WINDOWS = System.getProperty("os.name").contains("Windows"); - - @Rule - public ExpectedException exceptionRule = ExpectedException.none(); - - @Test - public void testEventMetadata() throws IOException { - PubSubToElasticsearchOptions options = - TestPipeline.testingPipelineOptions().as(PubSubToElasticsearchOptions.class); - - options.setErrorOutputTopic("projects/test/topics/test-error-topic"); - options.setApiKey("key"); - options.setDataset(Dataset.AUDIT); - options.setNamespace("test-namespace"); - options.setElasticsearchTemplateVersion("999.999.999"); - - String inputMessage = readInputMessage(INPUT_MESSAGE_FILE_PATH); - EventMetadataBuilder eventMetadataBuilder = EventMetadataBuilder.build(inputMessage, options); - JsonNode enrichedMessageAsJson = eventMetadataBuilder.getEnrichedMessageAsJsonNode(); - String enrichedMessageAsString = eventMetadataBuilder.getEnrichedMessageAsString(); - - Assert.assertTrue(StringUtils.isNotBlank(enrichedMessageAsString)); - Assert.assertEquals(inputMessage, enrichedMessageAsJson.get("message").textValue()); + private static final String RESOURCES_DIR = "EventMetadataBuilder/"; + private static final String INPUT_MESSAGE_FILE_PATH = + Resources.getResource(RESOURCES_DIR + "inputGCPAuditlogMessage.json").getPath(); + private static final String INPUT_MESSAGE_INVALID_FILE_PATH = + Resources.getResource(RESOURCES_DIR + "inputGCPAuditlogMessageInvalid.json").getPath(); + private static final boolean IS_WINDOWS = System.getProperty("os.name").contains("Windows"); + + @Rule public ExpectedException exceptionRule = ExpectedException.none(); + + @Test + public void testEventMetadata() throws IOException { + PubSubToElasticsearchOptions options = + TestPipeline.testingPipelineOptions().as(PubSubToElasticsearchOptions.class); + + options.setErrorOutputTopic("projects/test/topics/test-error-topic"); + options.setApiKey("key"); + options.setDataset(Dataset.AUDIT); + options.setNamespace("test-namespace"); + options.setElasticsearchTemplateVersion("999.999.999"); + + String inputMessage = readInputMessage(INPUT_MESSAGE_FILE_PATH); + EventMetadataBuilder eventMetadataBuilder = EventMetadataBuilder.build(inputMessage, options); + JsonNode enrichedMessageAsJson = eventMetadataBuilder.getEnrichedMessageAsJsonNode(); + String enrichedMessageAsString = eventMetadataBuilder.getEnrichedMessageAsString(); + + Assert.assertTrue(StringUtils.isNotBlank(enrichedMessageAsString)); + Assert.assertEquals(inputMessage, enrichedMessageAsJson.get("message").textValue()); Assert.assertEquals( "999.999.999", enrichedMessageAsJson.get("agent").get("version").textValue()); Assert.assertEquals( @@ -78,54 +76,56 @@ public void testEventMetadata() throws IOException { enrichedMessageAsJson.get("service").get("type").textValue()); Assert.assertEquals( "2021-07-14T10:35:17.528142Z", enrichedMessageAsJson.get("@timestamp").textValue()); - } + } - @Test - public void testEventMetadataAppend() throws IOException { - PubSubToElasticsearchOptions options = - TestPipeline.testingPipelineOptions().as(PubSubToElasticsearchOptions.class); + @Test + public void testEventMetadataAppend() throws IOException { + PubSubToElasticsearchOptions options = + TestPipeline.testingPipelineOptions().as(PubSubToElasticsearchOptions.class); - options.setErrorOutputTopic("projects/test/topics/test-error-topic"); - options.setApiKey("key"); - options.setDataset(Dataset.AUDIT); - options.setNamespace("test-namespace"); + options.setErrorOutputTopic("projects/test/topics/test-error-topic"); + options.setApiKey("key"); + options.setDataset(Dataset.AUDIT); + options.setNamespace("test-namespace"); - String inputMessage = readInputMessage(INPUT_MESSAGE_FILE_PATH); - EventMetadataBuilder eventMetadataBuilder = EventMetadataBuilder.build(inputMessage, options); - JsonNode enrichedMessageAsJson = eventMetadataBuilder.getEnrichedMessageAsJsonNode(); + String inputMessage = readInputMessage(INPUT_MESSAGE_FILE_PATH); + EventMetadataBuilder eventMetadataBuilder = EventMetadataBuilder.build(inputMessage, options); + JsonNode enrichedMessageAsJson = eventMetadataBuilder.getEnrichedMessageAsJsonNode(); - //if elasticsearchTemplateVersion is not set, 1.0.0 is the default value - Assert.assertEquals("1.0.0", enrichedMessageAsJson.get("agent").get("version").textValue()); + // if elasticsearchTemplateVersion is not set, 1.0.0 is the default value + Assert.assertEquals("1.0.0", enrichedMessageAsJson.get("agent").get("version").textValue()); Assert.assertEquals( enrichedMessageAsJson.get("data_stream").get("dataset").textValue(), Dataset.AUDIT.getKeyWithPrefix()); - } + } - @Test - public void testEventMetadataAppendFailed() throws IOException { - exceptionRule.expect(IllegalStateException.class); + @Test + public void testEventMetadataAppendFailed() throws IOException { + exceptionRule.expect(IllegalStateException.class); - PubSubToElasticsearchOptions options = - TestPipeline.testingPipelineOptions().as(PubSubToElasticsearchOptions.class); + PubSubToElasticsearchOptions options = + TestPipeline.testingPipelineOptions().as(PubSubToElasticsearchOptions.class); - options.setErrorOutputTopic("projects/test/topics/test-error-topic"); - options.setApiKey("key"); - options.setDataset(Dataset.AUDIT); - options.setNamespace("test-namespace"); + options.setErrorOutputTopic("projects/test/topics/test-error-topic"); + options.setApiKey("key"); + options.setDataset(Dataset.AUDIT); + options.setNamespace("test-namespace"); - String inputMessageInvalid = readInputMessage(INPUT_MESSAGE_INVALID_FILE_PATH); - EventMetadataBuilder eventMetadataBuilder = EventMetadataBuilder.build(inputMessageInvalid, options); - JsonNode enrichedMessageAsJson = eventMetadataBuilder.getEnrichedMessageAsJsonNode(); + String inputMessageInvalid = readInputMessage(INPUT_MESSAGE_INVALID_FILE_PATH); + EventMetadataBuilder eventMetadataBuilder = + EventMetadataBuilder.build(inputMessageInvalid, options); + JsonNode enrichedMessageAsJson = eventMetadataBuilder.getEnrichedMessageAsJsonNode(); - //if elasticsearchTemplateVersion is not set, 1.0.0 is the default value - Assert.assertEquals("1.0.0", enrichedMessageAsJson.get("agent").get("version").textValue()); + // if elasticsearchTemplateVersion is not set, 1.0.0 is the default value + Assert.assertEquals("1.0.0", enrichedMessageAsJson.get("agent").get("version").textValue()); Assert.assertEquals( enrichedMessageAsJson.get("data_stream").get("dataset").textValue(), Dataset.AUDIT.getKeyWithPrefix()); - } - - private String readInputMessage(String filePath) throws IOException { - return Files.lines(Paths.get(IS_WINDOWS ? filePath.substring(1) : filePath), StandardCharsets.UTF_8) - .collect(Collectors.joining()); - } -} \ No newline at end of file + } + + private String readInputMessage(String filePath) throws IOException { + return Files.lines( + Paths.get(IS_WINDOWS ? filePath.substring(1) : filePath), StandardCharsets.UTF_8) + .collect(Collectors.joining()); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/DynamicJdbcIO.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/DynamicJdbcIO.java index a298e25d65..dfbe66aa8d 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/DynamicJdbcIO.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/DynamicJdbcIO.java @@ -326,7 +326,7 @@ private void populateDisplayData(DisplayData.Builder builder) { public DataSource buildDatasource() { BasicDataSource basicDataSource = new BasicDataSource(); if (getDriverClassName() == null) { - throw new RuntimeException("Driver class name is required."); + throw new RuntimeException("Driver class name is required."); } else { basicDataSource.setDriverClassName(getDriverClassName()); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index f98bf880d2..89280d5a9a 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -174,8 +174,7 @@ static Pipeline buildPipeline( LOG.info("Loading BigQuery metadata..."); List tables = metadataLoader.loadDatasetMetadata( - datasetId, - new DataplexBigQueryToGcsFilter(options, existingTargetFiles)); + datasetId, new DataplexBigQueryToGcsFilter(options, existingTargetFiles)); LOG.info("Loaded {} table(s).", tables.size()); if (!tables.isEmpty()) { diff --git a/v2/googlecloud-to-googlecloud/src/main/java/org/apache/beam/sdk/io/jdbc/BeamSchemaUtil.java b/v2/googlecloud-to-googlecloud/src/main/java/org/apache/beam/sdk/io/jdbc/BeamSchemaUtil.java index 5aacd2f1d8..615c35b226 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/org/apache/beam/sdk/io/jdbc/BeamSchemaUtil.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/org/apache/beam/sdk/io/jdbc/BeamSchemaUtil.java @@ -23,7 +23,8 @@ * This class is used to expose the package level method {@link SchemaUtil#toBeamSchema} to the * Teleport project. * - * This util class is temporary until {@link SchemaUtil#toBeamSchema} is made public in Apache Beam. + *

This util class is temporary until {@link SchemaUtil#toBeamSchema} is made public in Apache + * Beam. */ public final class BeamSchemaUtil { From 5a88a3be78afada719ff483172f4a2f9ffc03256 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 5 Jan 2022 14:10:52 -0500 Subject: [PATCH 015/145] Add workflow for running spotless check --- .github/workflows/preconditions.yml | 51 +++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100644 .github/workflows/preconditions.yml diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml new file mode 100644 index 0000000000..dc84ef7b45 --- /dev/null +++ b/.github/workflows/preconditions.yml @@ -0,0 +1,51 @@ +# Copyright 2021 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 +# +# https://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. + +# Checks that don't run any template code and only on PRs. + +name: Precondition Checks + +on: + pull_request: + branches: ['main'] + +jobs: + spotless_check: + name: Spotless + timeout-minutes: 10 + runs-on: ubuntu-latest + steps: + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Java + uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 + with: + distribution: 'zulu' + version: '11' # Spotless won't work on version 8 + cache: 'maven' + - name: Run Spotless + run: | + function get_change_count { + echo $(git status | grep -e $1 | wc -l) + } + if [[ $(get_change_count '[^/]src/') -gt 0 ]]; then + mvn spotless:check + else + echo 'No changes to Classic Templates. Skipping spotless check' + fi + if [[ $(get_change_count '[^/]v2/') -gt 0 ]]; then + mvn spotless:check -f v2/pom.xml + else + echo 'No changes to Flex Templates. Skipping spotless check.' + fi From c0015d8ab621a5f4be486954675c222e42f8edc3 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 5 Jan 2022 15:55:59 -0500 Subject: [PATCH 016/145] Fix argument name --- .github/workflows/preconditions.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index dc84ef7b45..e9a8f0bfe5 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -32,7 +32,7 @@ jobs: uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 with: distribution: 'zulu' - version: '11' # Spotless won't work on version 8 + java-version: '11' # Spotless won't work on version 8 cache: 'maven' - name: Run Spotless run: | From 86797734a4710a983b3cf48cc893e2a8d2a8f1f6 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 5 Jan 2022 15:57:18 -0500 Subject: [PATCH 017/145] Rename the workflow --- .github/workflows/preconditions.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index e9a8f0bfe5..d80aa91f8e 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -14,7 +14,7 @@ # Checks that don't run any template code and only on PRs. -name: Precondition Checks +name: Preconditions on: pull_request: From 30cdc3e06ae3ba38a3ec6bd77c3693b137f06a20 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 6 Jan 2022 10:26:59 -0500 Subject: [PATCH 018/145] Move script to separate file for easier testing and reference --- .github/workflows/preconditions.yml | 15 +-------- scripts/workflows/run_spotless_check.sh | 44 +++++++++++++++++++++++++ 2 files changed, 45 insertions(+), 14 deletions(-) create mode 100755 scripts/workflows/run_spotless_check.sh diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index d80aa91f8e..a3db99134d 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -35,17 +35,4 @@ jobs: java-version: '11' # Spotless won't work on version 8 cache: 'maven' - name: Run Spotless - run: | - function get_change_count { - echo $(git status | grep -e $1 | wc -l) - } - if [[ $(get_change_count '[^/]src/') -gt 0 ]]; then - mvn spotless:check - else - echo 'No changes to Classic Templates. Skipping spotless check' - fi - if [[ $(get_change_count '[^/]v2/') -gt 0 ]]; then - mvn spotless:check -f v2/pom.xml - else - echo 'No changes to Flex Templates. Skipping spotless check.' - fi + run: ./scripts/workflows/run_spotless_check.sh diff --git a/scripts/workflows/run_spotless_check.sh b/scripts/workflows/run_spotless_check.sh new file mode 100755 index 0000000000..185f7be2bb --- /dev/null +++ b/scripts/workflows/run_spotless_check.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Copyright 2021 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 +# +# https://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. + +echo 'Running spotless to verify that the code sticks to the Google style guide.' + +FINAL_STATUS=0 + +# Attempts to get the changes based on a regex that filters from `git status`. +function get_change_count { + echo $(git status | grep -e $1 | wc -l) +} + +if [[ $(get_change_count '[^/]src/') -gt 0 ]]; then + mvn spotless:check + readonly CLASSIC_STATUS=$? + if [[ $CLASSIC_STATUS -ne 0 ]]; then echo 'Error in Classic Templates. Run `mvn spotless:apply` from root to fix'; fi + FINAL_STATUS=$(($FINAL_STATUS | $CLASSIC_STATUS)) +else + echo 'No changes detected in Classic Templates. Skipping spotless check.' +fi + +if [[ $(get_change_count '[^/]v2/') -gt 0 ]]; then + mvn spotless:check -f v2/pom.xml + readonly FLEX_STATUS=$? + if [[ $FLEX_STATUS -ne 0 ]]; then echo 'Error in Flex Templates. Run `mvn spotless:apply -f v2/pom.xml` to fix'; fi + FINAL_STATUS=$(($FINAL_STATUS | $FLEX_STATUS)) +else + echo 'No changes detected in Flex Templates. Skipping spotless check.' +fi + +echo 'Check complete.' +exit $FINAL_STATUS \ No newline at end of file From 365e9a51b35a8707aa502f0860cdeffd9e3dc344 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 6 Jan 2022 10:28:51 -0500 Subject: [PATCH 019/145] Add newline to script file --- scripts/workflows/run_spotless_check.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/workflows/run_spotless_check.sh b/scripts/workflows/run_spotless_check.sh index 185f7be2bb..87c5c4bc06 100755 --- a/scripts/workflows/run_spotless_check.sh +++ b/scripts/workflows/run_spotless_check.sh @@ -41,4 +41,4 @@ else fi echo 'Check complete.' -exit $FINAL_STATUS \ No newline at end of file +exit $FINAL_STATUS From 1f1ecc5e478c06f6a5f0292bf7f0795abace1238 Mon Sep 17 00:00:00 2001 From: Zachary Houfek <83302349+zhoufek@users.noreply.github.com> Date: Fri, 7 Jan 2022 15:25:08 -0500 Subject: [PATCH 020/145] Run Spotless on PRs (#322) Add workflow for running spotless check --- .github/workflows/preconditions.yml | 38 +++++++++++++++++++++ scripts/workflows/run_spotless_check.sh | 44 +++++++++++++++++++++++++ 2 files changed, 82 insertions(+) create mode 100644 .github/workflows/preconditions.yml create mode 100755 scripts/workflows/run_spotless_check.sh diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml new file mode 100644 index 0000000000..a3db99134d --- /dev/null +++ b/.github/workflows/preconditions.yml @@ -0,0 +1,38 @@ +# Copyright 2021 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 +# +# https://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. + +# Checks that don't run any template code and only on PRs. + +name: Preconditions + +on: + pull_request: + branches: ['main'] + +jobs: + spotless_check: + name: Spotless + timeout-minutes: 10 + runs-on: ubuntu-latest + steps: + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Java + uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 + with: + distribution: 'zulu' + java-version: '11' # Spotless won't work on version 8 + cache: 'maven' + - name: Run Spotless + run: ./scripts/workflows/run_spotless_check.sh diff --git a/scripts/workflows/run_spotless_check.sh b/scripts/workflows/run_spotless_check.sh new file mode 100755 index 0000000000..87c5c4bc06 --- /dev/null +++ b/scripts/workflows/run_spotless_check.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Copyright 2021 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 +# +# https://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. + +echo 'Running spotless to verify that the code sticks to the Google style guide.' + +FINAL_STATUS=0 + +# Attempts to get the changes based on a regex that filters from `git status`. +function get_change_count { + echo $(git status | grep -e $1 | wc -l) +} + +if [[ $(get_change_count '[^/]src/') -gt 0 ]]; then + mvn spotless:check + readonly CLASSIC_STATUS=$? + if [[ $CLASSIC_STATUS -ne 0 ]]; then echo 'Error in Classic Templates. Run `mvn spotless:apply` from root to fix'; fi + FINAL_STATUS=$(($FINAL_STATUS | $CLASSIC_STATUS)) +else + echo 'No changes detected in Classic Templates. Skipping spotless check.' +fi + +if [[ $(get_change_count '[^/]v2/') -gt 0 ]]; then + mvn spotless:check -f v2/pom.xml + readonly FLEX_STATUS=$? + if [[ $FLEX_STATUS -ne 0 ]]; then echo 'Error in Flex Templates. Run `mvn spotless:apply -f v2/pom.xml` to fix'; fi + FINAL_STATUS=$(($FINAL_STATUS | $FLEX_STATUS)) +else + echo 'No changes detected in Flex Templates. Skipping spotless check.' +fi + +echo 'Check complete.' +exit $FINAL_STATUS From 4fe10c8f7c32830af8f781099fb39a7576789909 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Fri, 7 Jan 2022 20:38:36 +0000 Subject: [PATCH 021/145] Copybara config change PiperOrigin-RevId: 420349407 --- .github/workflows/preconditions.yml | 38 --------------------- scripts/workflows/run_spotless_check.sh | 44 ------------------------- 2 files changed, 82 deletions(-) delete mode 100644 .github/workflows/preconditions.yml delete mode 100755 scripts/workflows/run_spotless_check.sh diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml deleted file mode 100644 index a3db99134d..0000000000 --- a/.github/workflows/preconditions.yml +++ /dev/null @@ -1,38 +0,0 @@ -# Copyright 2021 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 -# -# https://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. - -# Checks that don't run any template code and only on PRs. - -name: Preconditions - -on: - pull_request: - branches: ['main'] - -jobs: - spotless_check: - name: Spotless - timeout-minutes: 10 - runs-on: ubuntu-latest - steps: - - name: Checkout Code - uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 - - name: Setup Java - uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 - with: - distribution: 'zulu' - java-version: '11' # Spotless won't work on version 8 - cache: 'maven' - - name: Run Spotless - run: ./scripts/workflows/run_spotless_check.sh diff --git a/scripts/workflows/run_spotless_check.sh b/scripts/workflows/run_spotless_check.sh deleted file mode 100755 index 87c5c4bc06..0000000000 --- a/scripts/workflows/run_spotless_check.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/usr/bin/env bash -# Copyright 2021 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 -# -# https://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. - -echo 'Running spotless to verify that the code sticks to the Google style guide.' - -FINAL_STATUS=0 - -# Attempts to get the changes based on a regex that filters from `git status`. -function get_change_count { - echo $(git status | grep -e $1 | wc -l) -} - -if [[ $(get_change_count '[^/]src/') -gt 0 ]]; then - mvn spotless:check - readonly CLASSIC_STATUS=$? - if [[ $CLASSIC_STATUS -ne 0 ]]; then echo 'Error in Classic Templates. Run `mvn spotless:apply` from root to fix'; fi - FINAL_STATUS=$(($FINAL_STATUS | $CLASSIC_STATUS)) -else - echo 'No changes detected in Classic Templates. Skipping spotless check.' -fi - -if [[ $(get_change_count '[^/]v2/') -gt 0 ]]; then - mvn spotless:check -f v2/pom.xml - readonly FLEX_STATUS=$? - if [[ $FLEX_STATUS -ne 0 ]]; then echo 'Error in Flex Templates. Run `mvn spotless:apply -f v2/pom.xml` to fix'; fi - FINAL_STATUS=$(($FINAL_STATUS | $FLEX_STATUS)) -else - echo 'No changes detected in Flex Templates. Skipping spotless check.' -fi - -echo 'Check complete.' -exit $FINAL_STATUS From 46388f9cbea78eac1d54462f337c658b433821d4 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Mon, 10 Jan 2022 09:52:52 -0800 Subject: [PATCH 022/145] Update http-client to 1.40.1 PiperOrigin-RevId: 420786677 --- pom.xml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c7e9ae42a0..90d1d59720 100644 --- a/pom.xml +++ b/pom.xml @@ -36,7 +36,7 @@ UTF-8 - 1.39.2 + 1.40.1 1.0-rc6 1.8.2 8.7 @@ -208,6 +208,11 @@ + + com.google.http-client + google-http-client + ${apache-http-client-v2} + com.google.http-client google-http-client-apache-v2 From 8e75e275ddde19cd5452a691e00e661e31e0d508 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Mon, 10 Jan 2022 14:26:40 -0800 Subject: [PATCH 023/145] Removing default value for firestoreHintNumWorkers parameter. firestoreHintNumWorkers and datastoreHintNumWorkers have default values of 500. Either one can be set by the user. After this change, FirestoreNestedValueProvider will automatically select the user provided value over the default value. PiperOrigin-RevId: 420856978 --- .../cloud/teleport/templates/common/DatastoreConverters.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java index 8df19100e5..54ee00bc9f 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/DatastoreConverters.java @@ -173,7 +173,6 @@ public interface DatastoreWriteOptions extends PipelineOptions { void setFirestoreWriteNamespace(ValueProvider value); @Description("Hint for the expected number of workers in the ramp-up throttling step") - @Default.Integer(500) ValueProvider getFirestoreHintNumWorkers(); void setFirestoreHintNumWorkers(ValueProvider value); @@ -210,7 +209,6 @@ public interface DatastoreDeleteOptions extends PipelineOptions { void setFirestoreDeleteProjectId(ValueProvider firestoreDeleteProjectId); @Description("Hint for the expected number of workers in the ramp-up throttling step") - @Default.Integer(500) ValueProvider getFirestoreHintNumWorkers(); void setFirestoreHintNumWorkers(ValueProvider value); From 17839a9630574f15b2bcfcf44af856d481526c84 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 11:05:32 -0500 Subject: [PATCH 024/145] Rewrite the spotless check so that some parts are reusable --- cicd/cmd/run-spotless.go | 66 +++++++++++++++ cicd/op/maven.go | 36 +++++++++ cicd/op/run-cmd.go | 41 ++++++++++ cicd/repo/modules.go | 163 ++++++++++++++++++++++++++++++++++++++ cicd/repo/modules_test.go | 78 ++++++++++++++++++ 5 files changed, 384 insertions(+) create mode 100644 cicd/cmd/run-spotless.go create mode 100644 cicd/op/maven.go create mode 100644 cicd/op/run-cmd.go create mode 100644 cicd/repo/modules.go create mode 100644 cicd/repo/modules_test.go diff --git a/cicd/cmd/run-spotless.go b/cicd/cmd/run-spotless.go new file mode 100644 index 0000000000..9994c5f823 --- /dev/null +++ b/cicd/cmd/run-spotless.go @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2021 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 main + +import ( + "flag" + "fmt" + "log" + "strings" + + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/op" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/repo" +) + +const ( + SpotlessCommand = "spotless:check" +) + +func main() { + changed := flag.String("changed-files", "", "List of changed files as a comma-separated string") + flag.Parse() + + if len(*changed) == 0 { + log.Print("No changed files passed. This is probably an error, but we're assuming it isn't just in case") + return + } + + s := strings.Split(*changed, ",") + modules := repo.GetModulesForPaths(s) + + var fullErr error + if _, ok := modules[repo.ClassicRoot]; ok { + if err := op.RunMavenOnPom(repo.ClassicRoot, SpotlessCommand); err != nil { + fullErr = err + } + } + if flex, ok := modules[repo.FlexRoot]; ok { + for _, m := range flex { + if len(m) > 0 { // Zero length represents the root, which has no Java code + if err := op.RunMavenOnModule(repo.FlexRoot, SpotlessCommand, m); err != nil && fullErr != nil { + fullErr = err + } else if err != nil { + fullErr = fmt.Errorf("%w\n%v", fullErr, err) + } + } + } + } + + if fullErr != nil { + log.Fatal("There were spotless errors. Check the output from the commands.") + } +} diff --git a/cicd/op/maven.go b/cicd/op/maven.go new file mode 100644 index 0000000000..47ac88a27b --- /dev/null +++ b/cicd/op/maven.go @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2021 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 op + +import ( + "fmt" +) + +// Run the given Maven command on a specified POM file. Considering the input, this is equivalent to: +// mvn {cmd} -f {pomDir}/pom.xml {args...} +func RunMavenOnPom(pomDir string, cmd string, args ...string) error { + wa := []string{cmd, "-f", fmt.Sprintf("%s/pom.xml", pomDir)} + return RunCmdAndStreamOutput("mvn", append(wa, args...)) +} + +// Rung the given Maven command on a specified module. Considering the input, this is equivalent to: +// mvn {cmd} -f {pomDir}/pom.xml -pl {module} {args...} +func RunMavenOnModule(pomDir string, cmd string, module string, args ...string) error { + fullArgs := []string{"-pl", module} + fullArgs = append(fullArgs, args...) + return RunMavenOnPom(pomDir, cmd, fullArgs...) +} diff --git a/cicd/op/run-cmd.go b/cicd/op/run-cmd.go new file mode 100644 index 0000000000..44976e1e50 --- /dev/null +++ b/cicd/op/run-cmd.go @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2021 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 op + +import ( + "bufio" + "fmt" + "log" + "os/exec" + "strings" +) + +// Runs a command and streams the output rather than waiting for it to complete. +func RunCmdAndStreamOutput(cmd string, args []string) error { + log.Printf("Running command: %s %s", cmd, strings.Join(args, " ")) + op := exec.Command(cmd, args...) + + stdout, _ := op.StdoutPipe() + op.Start() + + scanner := bufio.NewScanner(stdout) + for scanner.Scan() { + fmt.Println(scanner.Text()) + } + + return op.Wait() +} diff --git a/cicd/repo/modules.go b/cicd/repo/modules.go new file mode 100644 index 0000000000..27f2431a6a --- /dev/null +++ b/cicd/repo/modules.go @@ -0,0 +1,163 @@ +/* + * Copyright (C) 2021 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 repo + +import ( + "fmt" + "os" + "strings" +) + +const ( + // Roots in relation to the root directory of the repository. + ClassicRoot = "." + FlexRoot = "v2" +) + +// Returns a map of roots to their modules. Properties are: +// Key: The root module, equivalent to one of the const values (e.g. ClassicRoot) +// Value: All the submodules, sometimes nested under another parent that is also in the slice +// This could be used in the equivalent command: +// mvn x:y -f {key}/pom.xml -pl {value} +// An empty value indicates no submodules. +func GetModuleMapping() map[string][]string { + return map[string][]string{ + ClassicRoot: []string{}, + FlexRoot: []string{ + "bigquery-to-bigtable", + "bigquery-to-parquet", + "cdc-parent/cdc-embedded-connector", + "cdc-parent/cdc-common", + "cdc-parent", + "cdc-parent/cdc-agg", + "cdc-parent/cdc-change-applier", + "common", + "datastream-to-sql", + "datastream-to-bigquery", + "datastream-to-mongodb", + "datastream-to-postgres", + "datastream-to-spanner", + "elasticsearch-common", + "file-format-conversion", + "googlecloud-to-googlecloud", + "googlecloud-to-elasticsearch", + "hive-to-bigquery", + "kafka-to-bigquery", + "kafka-common", + "kafka-to-gcs", + "kafka-to-pubsub", + "kudu-to-bigquery", + "pubsub-binary-to-bigquery", + "pubsub-cdc-to-bigquery", + "pubsub-to-mongodb", + }, + } +} + +type moduleTrieNode struct { + value string + children map[rune]*moduleTrieNode +} + +func flexModulesAsTrie() *moduleTrieNode { + root := &moduleTrieNode{ + value: "", + children: make(map[rune]*moduleTrieNode), + } + + for _, m := range GetModuleMapping()[FlexRoot] { + curr := root + for _, r := range m { + if _, ok := curr.children[r]; ok { + curr = curr.children[r] + } else { + curr.children[r] = &moduleTrieNode{ + value: "", + children: make(map[rune]*moduleTrieNode), + } + curr = curr.children[r] + } + } + curr.value = m + } + + return root +} + +func findUniqueFlexModules(paths []string) []string { + trie := flexModulesAsTrie() + modules := make(map[string]interface{}) + + for _, path := range paths { + curr := trie + var possible *moduleTrieNode + + for _, r := range path { + var ok bool + curr, ok = curr.children[r] + if !ok { + break + } + if curr.value != "" { + possible = curr + } + } + + if possible != nil { + modules[possible.value] = nil + } + // We don't error from not finding anything, since it could be a root-level file + // that isn't part of any module. + } + + ret := make([]string, len(modules)) + i := 0 + for k := range modules { + ret[i] = k + i += 1 + } + + return ret +} + +func GetModulesForPaths(paths []string) map[string][]string { + if len(paths) == 0 { + return make(map[string][]string) + } + + m := make(map[string][]string) + flex := make([]string, 0) + + v2 := fmt.Sprintf("v2%s", string(os.PathSeparator)) + + for _, path := range paths { + if strings.HasPrefix(path, v2) { + flex = append(flex, strings.TrimPrefix(path, v2)) + } else { + // TODO(zhoufek): Make this more granular, especially separating .github and cicd code + // into separate "modules" + m[ClassicRoot] = make([]string, 0) + } + } + + if len(flex) > 0 { + // Even if nothing is found, we should still account for v2/ as its own module, since + m[FlexRoot] = findUniqueFlexModules(flex) + } + + return m +} diff --git a/cicd/repo/modules_test.go b/cicd/repo/modules_test.go new file mode 100644 index 0000000000..ec2014505d --- /dev/null +++ b/cicd/repo/modules_test.go @@ -0,0 +1,78 @@ +/* + * Copyright (C) 2021 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 repo + +import ( + "reflect" + "testing" +) + +func TestModuleMappingHasAllRoots(t *testing.T) { + m := GetModuleMapping() + if _, ok := m[ClassicRoot]; !ok { + t.Error("Missing Classic root") + } + if _, ok := m[FlexRoot]; !ok { + t.Error("Missing Flex root") + } +} + +func TestGetModulesForPaths(t *testing.T) { + tests := []struct { + input []string + expected map[string][]string + }{ + { + input: []string{"src/something", "v2/pubsub-binary-to-bigquery/avro", "src/something-else", "v2/pubsub-binary-to-bigquery/proto"}, + expected: map[string][]string{ + ClassicRoot: []string{}, + FlexRoot: []string{"pubsub-binary-to-bigquery"}, + }, + }, + { + input: []string{"v2/pubsub-cdc-to-bigquery", "v2/pubsub-binary-to-bigquery"}, + expected: map[string][]string{ + FlexRoot: []string{"pubsub-cdc-to-bigquery", "pubsub-binary-to-bigquery"}, + }, + }, + { + input: []string{"v2/cdc-parent/something", "v2/cdc-parent/cdc-common/something"}, + expected: map[string][]string{ + FlexRoot: []string{"cdc-parent", "cdc-parent/cdc-common"}, + }, + }, + { + input: []string{"something", "v2/something"}, + expected: map[string][]string{ + ClassicRoot: make([]string, 0), + FlexRoot: make([]string, 0), + }, + }, + { + input: make([]string, 0), + expected: make(map[string][]string), + }, + } + + for _, test := range tests { + t.Logf("Testing input: %v", test.input) + if actual := GetModulesForPaths(test.input); !reflect.DeepEqual(actual, test.expected) { + t.Errorf("incorrect results. expected: %v. got: %v", test.expected, actual) + } + t.Logf("Success") + } +} From b51f973769250f356646b6a0dd94f9e9bd4b8d91 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 11:39:46 -0500 Subject: [PATCH 025/145] Update the GitHub Action for running spotless --- .github/workflows/preconditions.yml | 8 +++++++- cicd/cmd/run-spotless.go | 1 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index a3db99134d..74a14172b3 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -34,5 +34,11 @@ jobs: distribution: 'zulu' java-version: '11' # Spotless won't work on version 8 cache: 'maven' + - name: Setup Go + uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf + - name: Get Changed Files + uses: tj-actions/changed-files@61ee456a9d0f512e7ecfdf28863634c97dae2d16 + with: + separator: ',' - name: Run Spotless - run: ./scripts/workflows/run_spotless_check.sh + run: go run ./cicd/cmd/run-spotless.go --changed-files="${{ steps.changed-files.outputs.all_changed_and_modified_files }}" diff --git a/cicd/cmd/run-spotless.go b/cicd/cmd/run-spotless.go index 9994c5f823..207d5b40e3 100644 --- a/cicd/cmd/run-spotless.go +++ b/cicd/cmd/run-spotless.go @@ -38,6 +38,7 @@ func main() { log.Print("No changed files passed. This is probably an error, but we're assuming it isn't just in case") return } + log.Printf("Received changed files: %s", *changed) s := strings.Split(*changed, ",") modules := repo.GetModulesForPaths(s) From d43eef7551fe257cef53c652adba57c99deda6b8 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 11:40:28 -0500 Subject: [PATCH 026/145] Remove the original script for running Spotless --- scripts/workflows/run_spotless_check.sh | 44 ------------------------- 1 file changed, 44 deletions(-) delete mode 100755 scripts/workflows/run_spotless_check.sh diff --git a/scripts/workflows/run_spotless_check.sh b/scripts/workflows/run_spotless_check.sh deleted file mode 100755 index 87c5c4bc06..0000000000 --- a/scripts/workflows/run_spotless_check.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/usr/bin/env bash -# Copyright 2021 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 -# -# https://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. - -echo 'Running spotless to verify that the code sticks to the Google style guide.' - -FINAL_STATUS=0 - -# Attempts to get the changes based on a regex that filters from `git status`. -function get_change_count { - echo $(git status | grep -e $1 | wc -l) -} - -if [[ $(get_change_count '[^/]src/') -gt 0 ]]; then - mvn spotless:check - readonly CLASSIC_STATUS=$? - if [[ $CLASSIC_STATUS -ne 0 ]]; then echo 'Error in Classic Templates. Run `mvn spotless:apply` from root to fix'; fi - FINAL_STATUS=$(($FINAL_STATUS | $CLASSIC_STATUS)) -else - echo 'No changes detected in Classic Templates. Skipping spotless check.' -fi - -if [[ $(get_change_count '[^/]v2/') -gt 0 ]]; then - mvn spotless:check -f v2/pom.xml - readonly FLEX_STATUS=$? - if [[ $FLEX_STATUS -ne 0 ]]; then echo 'Error in Flex Templates. Run `mvn spotless:apply -f v2/pom.xml` to fix'; fi - FINAL_STATUS=$(($FINAL_STATUS | $FLEX_STATUS)) -else - echo 'No changes detected in Flex Templates. Skipping spotless check.' -fi - -echo 'Check complete.' -exit $FINAL_STATUS From 6b9d0533680e15a111faca2fc4826e3e08c418cb Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 12:27:30 -0500 Subject: [PATCH 027/145] Restructure project --- .github/workflows/preconditions.yml | 5 ++++- cicd/cmd/{run-spotless.go => run-spotless/main.go} | 4 ++-- cicd/go.mod | 3 +++ cicd/{ => internal}/op/maven.go | 0 cicd/{ => internal}/op/run-cmd.go | 0 cicd/{ => internal}/repo/modules.go | 0 cicd/{ => internal}/repo/modules_test.go | 0 7 files changed, 9 insertions(+), 3 deletions(-) rename cicd/cmd/{run-spotless.go => run-spotless/main.go} (92%) create mode 100644 cicd/go.mod rename cicd/{ => internal}/op/maven.go (100%) rename cicd/{ => internal}/op/run-cmd.go (100%) rename cicd/{ => internal}/repo/modules.go (100%) rename cicd/{ => internal}/repo/modules_test.go (100%) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index 74a14172b3..abf8b7fbc1 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -36,9 +36,12 @@ jobs: cache: 'maven' - name: Setup Go uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf + with: + go-version: '1.17' - name: Get Changed Files + id: changed-files uses: tj-actions/changed-files@61ee456a9d0f512e7ecfdf28863634c97dae2d16 with: separator: ',' - name: Run Spotless - run: go run ./cicd/cmd/run-spotless.go --changed-files="${{ steps.changed-files.outputs.all_changed_and_modified_files }}" + run: go run ./cicd/cmd/run-spotless.go --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" diff --git a/cicd/cmd/run-spotless.go b/cicd/cmd/run-spotless/main.go similarity index 92% rename from cicd/cmd/run-spotless.go rename to cicd/cmd/run-spotless/main.go index 207d5b40e3..9b39ec42bd 100644 --- a/cicd/cmd/run-spotless.go +++ b/cicd/cmd/run-spotless/main.go @@ -22,8 +22,8 @@ import ( "log" "strings" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/op" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/repo" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" ) const ( diff --git a/cicd/go.mod b/cicd/go.mod new file mode 100644 index 0000000000..6c6c8155c4 --- /dev/null +++ b/cicd/go.mod @@ -0,0 +1,3 @@ +module github.com/GoogleCloudPlatform/DataflowTemplates/cicd + +go 1.17 diff --git a/cicd/op/maven.go b/cicd/internal/op/maven.go similarity index 100% rename from cicd/op/maven.go rename to cicd/internal/op/maven.go diff --git a/cicd/op/run-cmd.go b/cicd/internal/op/run-cmd.go similarity index 100% rename from cicd/op/run-cmd.go rename to cicd/internal/op/run-cmd.go diff --git a/cicd/repo/modules.go b/cicd/internal/repo/modules.go similarity index 100% rename from cicd/repo/modules.go rename to cicd/internal/repo/modules.go diff --git a/cicd/repo/modules_test.go b/cicd/internal/repo/modules_test.go similarity index 100% rename from cicd/repo/modules_test.go rename to cicd/internal/repo/modules_test.go From a1c5efecd4c60ca829cf94ae1de8ef7b3f3a6649 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 12:28:54 -0500 Subject: [PATCH 028/145] Fix execution path --- .github/workflows/preconditions.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index abf8b7fbc1..9e7cd7dca5 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -44,4 +44,4 @@ jobs: with: separator: ',' - name: Run Spotless - run: go run ./cicd/cmd/run-spotless.go --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" + run: go run ./cicd/cmd/run-spotless --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" From 99864039fbe3afd8520054c2f2b28e58a57e70b0 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 12:50:00 -0500 Subject: [PATCH 029/145] Switch to building rather than using go run --- .github/workflows/preconditions.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index 9e7cd7dca5..582d37def4 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -44,4 +44,6 @@ jobs: with: separator: ',' - name: Run Spotless - run: go run ./cicd/cmd/run-spotless --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" + run: | + go build ./cicd/cmd/run-spotless + ./run-spotless --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" From 46e3c6c65e679e6a51291faf69c9dedf1ad1d565 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 12:53:32 -0500 Subject: [PATCH 030/145] Build from the actual root module --- .github/workflows/preconditions.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index 582d37def4..224ef4db70 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -45,5 +45,5 @@ jobs: separator: ',' - name: Run Spotless run: | - go build ./cicd/cmd/run-spotless - ./run-spotless --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" + cd cicd/ && go build ./cmd/run-spotless && cd .. + ./cicd/run-spotless --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" From 96bba0cc0e39f5cd93cdd0c39e3cbc1d6d59515d Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 12 Jan 2022 16:30:22 -0500 Subject: [PATCH 031/145] Fix and add comments --- cicd/internal/repo/modules.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index 27f2431a6a..e09c244c0a 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -134,6 +134,12 @@ func findUniqueFlexModules(paths []string) []string { return ret } +// Gets all the unique modules for files whose path from the root directory is in `paths`. Example paths: +// pom.xml -> Mapped to Classic root +// v2/cdc-parent/pom.xml -> Mapped to cdc-parent under Flex Templates +// The return value has the following properties: +// Key: The path of the root module, equivalent to ClassicRoot, FlexRoot, etc. +// Value: List of modules (e.g. cdc-parent, cdc-parent/cdc-common). An empty entry represents the root itself. func GetModulesForPaths(paths []string) map[string][]string { if len(paths) == 0 { return make(map[string][]string) @@ -156,6 +162,7 @@ func GetModulesForPaths(paths []string) map[string][]string { if len(flex) > 0 { // Even if nothing is found, we should still account for v2/ as its own module, since + // changes made be made to important files, like v2/pom.xml m[FlexRoot] = findUniqueFlexModules(flex) } From ee6457b308bfd75c3487dc71b42878679ce82f50 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 13 Jan 2022 12:54:44 -0800 Subject: [PATCH 032/145] Duplicate the sourceDirectory value for Avro plugin. This fixes some problems with building from IntelliJ. PiperOrigin-RevId: 421638327 --- pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pom.xml b/pom.xml index 90d1d59720..92168fec84 100644 --- a/pom.xml +++ b/pom.xml @@ -865,6 +865,11 @@ org.apache.avro avro-maven-plugin ${avro.version} + + + ${project.basedir}/src/main/resources/schema/avro + generate-sources From 79aca9d486ec3210ecf5444a0a33a3c4a77f819a Mon Sep 17 00:00:00 2001 From: WeiwenXu21 Date: Wed, 12 Jan 2022 15:02:14 -0500 Subject: [PATCH 033/145] Added issue templates --- .github/ISSUE_TEMPLATE/bug-template.yml | 88 +++++++++++++++++++ .github/ISSUE_TEMPLATE/config.yml | 1 + .../feature-request-template.yml | 65 ++++++++++++++ 3 files changed, 154 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/bug-template.yml create mode 100644 .github/ISSUE_TEMPLATE/config.yml create mode 100644 .github/ISSUE_TEMPLATE/feature-request-template.yml diff --git a/.github/ISSUE_TEMPLATE/bug-template.yml b/.github/ISSUE_TEMPLATE/bug-template.yml new file mode 100644 index 0000000000..b771c825e9 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/bug-template.yml @@ -0,0 +1,88 @@ +name: Bug +description: File a bug issue +title: "[Bug]: " +labels: ["bug", "p2", "needs triage"] +body: + - type: markdown + attributes: + value: | + Thank you so much for taking the time to fill out this bug report! Please note that changing the issue label from p2 to p0/p1 will *NOT* speed up triaging. **Any issue that is currently affecting your production should be directed to Google Cloud customer support.** If you cannot find the template in the dropdown menu below, we probably no longer support the template. + - type: dropdown + id: template + attributes: + label: Related Template + options: + - Pubsub-to-BigQuery (classic, streaming) + - PubsubAvro-to-BigQuery (flex, steaming) + - PubsubProto-to-BigQuery (flex, streaming) + - Pubsub-to-Pubsub (classic, streaming) + - Pubsub-to-Splunk (classic, streaming) + - Pubsub-to-Avro (classic, streaming) + - Pubsub-to-Text (classic, streaming) + - Pubsub-to-MongoDB (flex, streaming) + - Pubsub-to-Elasticsearch (flex, streaming) + - Datastream-to-Spanner (flex, streaming) + - Text-to-BigQuery-stream (classic, streaming) + - Text-to-Pubsub-stream (classic, streaming) + - DLPText-to-BigQuery-stream (classic, streaming) + - CDC (flex, streaming) + - Kafka-to-BigQuery (flex, streaming) + - Datastream-to-BigQuery (flex, streaming) + - Datastream-to-Postgres (flex, streaming) + - JDBC-to-Pubsub (flex, streaming) + - Pubsub-to-JDBC (flex, streaming) + - BigQuery-to-TFRecord (classic, batch) + - BigQuery-to-Parquet (flex, batch) + - BigQuery-to-Elasticsearch (flex, batch) + - Bigtable-to-Avro (classic, batch) + - Bigtable-to-Parquet (classic, batch) + - Datastore-to-Text (classic, batch) + - Spanner-to-Avro/Spanner-Export (classic, batch) + - Spanner-to-Text (classic, batch) + - Avro-to-Bigtable (classic, batch) + - Avro-to-Spanner/Spanner-Import (classic, batch) + - Parquet-to-Bigtable (classic, batch) + - TextIO-to-BigQuery (classic, batch) + - Text-to-Datastore (classic, batch) + - Text-to-Pubsub-batch (classic, batch) + - Text-to-Spanner/Spanner-TextImport (classic, batch) + - GCS-to-Elasticsearch (flex, batch) + - JDBC-to-BigQuery (classic, batch) + - Cassandra-to-Bigtable (classic, batch) + - File-Format-Conversion (flex, util) + - Bulk-Compressor (classic, util) + - Bulk-Decompressor (classic, util) + - Firestore-Bulk-Delete (classic, util) + - Streaming-Data-Generator (flex, util) + validations: + required: true + - type: dropdown + id: version + attributes: + label: Beam Version + description: What version of beam are you using? + options: + - 2.35.0 + - 2.34.0 + - 2.33.0 + - 2.32.0 + - 2.31.0 + - 2.30.0 + - Older than 2.30.0 + validations: + required: true + - type: textarea + id: what-happened + attributes: + label: What happened? + description: Please share what you expected and what happened! + placeholder: Tell us what you see! + validations: + required: true + - type: textarea + id: logs + attributes: + label: Relevant log output + description: Please copy and paste any relevant log output. This will be automatically formatted into code, so no need for backticks. + render: shell + diff --git a/.github/ISSUE_TEMPLATE/config.yml b/.github/ISSUE_TEMPLATE/config.yml new file mode 100644 index 0000000000..3ba13e0cec --- /dev/null +++ b/.github/ISSUE_TEMPLATE/config.yml @@ -0,0 +1 @@ +blank_issues_enabled: false diff --git a/.github/ISSUE_TEMPLATE/feature-request-template.yml b/.github/ISSUE_TEMPLATE/feature-request-template.yml new file mode 100644 index 0000000000..f52a8ea48b --- /dev/null +++ b/.github/ISSUE_TEMPLATE/feature-request-template.yml @@ -0,0 +1,65 @@ +name: Feature Request +description: File a feature request issue +title: "[Feature Request]: " +labels: ["addition", "p2", "needs triage"] +body: + - type: markdown + attributes: + value: | + Thank you so much for taking the time to fill out this feature request report! Please note that changing the issue label from p2 to p0/p1 will *NOT* speed up triaging. **Any issue that is currently affecting your production should be directed to Google Cloud customer support.** If you cannot find the template in the dropdown menu below, we probably no longer support the template. + - type: dropdown + id: template + attributes: + label: Related Template + options: + - Request a new template + - Pubsub-to-BigQuery (classic, streaming) + - PubsubAvro-to-BigQuery (flex, steaming) + - PubsubProto-to-BigQuery (flex, streaming) + - Pubsub-to-Pubsub (classic, streaming) + - Pubsub-to-Splunk (classic, streaming) + - Pubsub-to-Avro (classic, streaming) + - Pubsub-to-Text (classic, streaming) + - Pubsub-to-MongoDB (flex, streaming) + - Pubsub-to-Elasticsearch (flex, streaming) + - Datastream-to-Spanner (flex, streaming) + - Text-to-BigQuery-stream (classic, streaming) + - Text-to-Pubsub-stream (classic, streaming) + - DLPText-to-BigQuery-stream (classic, streaming) + - CDC (flex, streaming) + - Kafka-to-BigQuery (flex, streaming) + - Datastream-to-BigQuery (flex, streaming) + - Datastream-to-Postgres (flex, streaming) + - JDBC-to-Pubsub (flex, streaming) + - Pubsub-to-JDBC (flex, streaming) + - BigQuery-to-TFRecord (classic, batch) + - BigQuery-to-Parquet (flex, batch) + - BigQuery-to-Elasticsearch (flex, batch) + - Bigtable-to-Avro (classic, batch) + - Bigtable-to-Parquet (classic, batch) + - Datastore-to-Text (classic, batch) + - Spanner-to-Avro/Spanner-Export (classic, batch) + - Spanner-to-Text (classic, batch) + - Avro-to-Bigtable (classic, batch) + - Avro-to-Spanner/Spanner-Import (classic, batch) + - Parquet-to-Bigtable (classic, batch) + - TextIO-to-BigQuery (classic, batch) + - Text-to-Datastore (classic, batch) + - Text-to-Pubsub-batch (classic, batch) + - Text-to-Spanner/Spanner-TextImport (classic, batch) + - GCS-to-Elasticsearch (flex, batch) + - JDBC-to-BigQuery (classic, batch) + - Cassandra-to-Bigtable (classic, batch) + - File-Format-Conversion (flex, util) + - Bulk-Compressor (classic, util) + - Bulk-Decompressor (classic, util) + - Firestore-Bulk-Delete (classic, util) + - Streaming-Data-Generator (flex, util) + validations: + required: true + - type: textarea + id: what-happened + attributes: + label: What feature(s) are you requesting? + validations: + required: true From 1215436bb9f5256972f4a99b8893d4bf63f9220b Mon Sep 17 00:00:00 2001 From: zhoufek Date: Fri, 14 Jan 2022 14:48:35 -0500 Subject: [PATCH 034/145] Automate detecting modules --- cicd/internal/repo/files.go | 86 +++++++++++++++++++++++++++++++ cicd/internal/repo/files_test.go | 88 ++++++++++++++++++++++++++++++++ 2 files changed, 174 insertions(+) create mode 100644 cicd/internal/repo/files.go create mode 100644 cicd/internal/repo/files_test.go diff --git a/cicd/internal/repo/files.go b/cicd/internal/repo/files.go new file mode 100644 index 0000000000..539e8bdf8e --- /dev/null +++ b/cicd/internal/repo/files.go @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2021 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 repo + +import ( + "errors" + "fmt" + "io/fs" + "os" + "path/filepath" + "runtime" + "strings" +) + +const ( + RootDirName = "DataflowTemplates" +) + +func getRootDir() (string, error) { + _, path, _, ok := runtime.Caller(0) + if !ok { + return "", errors.New("could not determine a starting path to get to root directory") + } + + dir := filepath.Dir(path) + allDirs := strings.Split(dir, string(os.PathSeparator)) + i := len(allDirs) + for ; i >= 0 && allDirs[i-1] != RootDirName; i -= 1 { + // Empty intentionally + } + + if i == 0 { + return "", fmt.Errorf("%s is not in the %s project somehow", dir, RootDirName) + } + + return strings.Join(allDirs[:i], string(os.PathSeparator)), nil +} + +// Gets all the POM files under `dir`. `dir` is a relative path from the root of the repository. +// So if the root is located at `$HOME/go/src/github.com/GoogleCloudPlatform/DataflowTemplates`, then +// passing `v2` represents `$HOME/go/src/github.com/GoogleCloudPlatform/DataflowTemplates/v2`. +func GetAllPomFiles(dir string) ([]string, error) { + root, e := getRootDir() + if e != nil { + return nil, e + } + poms := make([]string, 0) + + var start string + if strings.HasPrefix(dir, RootDirName) { + start = root + } else { + start = filepath.Join(root, dir) + } + + e = filepath.Walk(start, func(path string, info fs.FileInfo, err error) error { + if err != nil { + return err + } + if info.IsDir() || info.Name() != "pom.xml" { + return nil + } + + poms = append(poms, path) + return nil + }) + + if e != nil { + return nil, e + } + return poms, nil +} diff --git a/cicd/internal/repo/files_test.go b/cicd/internal/repo/files_test.go new file mode 100644 index 0000000000..45e421aec4 --- /dev/null +++ b/cicd/internal/repo/files_test.go @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2021 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 repo + +import ( + "os" + "path/filepath" + "strings" + "testing" +) + +func TestGetAllPomFiles(t *testing.T) { + getModule := func(pomFile string) string { + dir := filepath.Dir(pomFile) + return dir[strings.LastIndex(dir, string(os.PathSeparator))+1:] + } + + tests := []struct { + dir string + expected map[string]interface{} + }{ + { + dir: FlexRoot, + expected: map[string]interface{}{ + "v2": nil, + "cdc-parent": nil, + "cdc-common": nil, + "pubsub-binary-to-bigquery": nil, + "googlecloud-to-googlecloud": nil, + }, + }, + { + dir: RootDirName, + expected: map[string]interface{}{ + RootDirName: nil, + "v2": nil, + "cdc-parent": nil, + "cdc-common": nil, + "pubsub-binary-to-bigquery": nil, + "googlecloud-to-googlecloud": nil, + }, + }, + { + dir: ClassicRoot, + expected: map[string]interface{}{ + RootDirName: nil, + "v2": nil, + "cdc-parent": nil, + "cdc-common": nil, + "pubsub-binary-to-bigquery": nil, + "googlecloud-to-googlecloud": nil, + }, + }, + } + + for _, test := range tests { + t.Logf("testing %s", test.dir) + + files, err := GetAllPomFiles(test.dir) + if err != nil { + t.Fatalf("error getting POM files: %v", err) + } + + for _, f := range files { + delete(test.expected, getModule(f)) + } + + if len(test.expected) > 0 { + t.Fatalf("did not encounter %v. got %v", test.expected, files) + } + + t.Logf("successful for %s", test.dir) + } +} From 9f098704dfb2f05dc9cf1ade4f982a43cb2f35d5 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Fri, 14 Jan 2022 15:14:35 -0500 Subject: [PATCH 035/145] Switch to generating modules rather than hardcoding them --- cicd/internal/repo/modules.go | 69 ++++++++++++++++++++--------------- 1 file changed, 39 insertions(+), 30 deletions(-) diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index e09c244c0a..1d124f3b3d 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -19,6 +19,7 @@ package repo import ( "fmt" "os" + "path/filepath" "strings" ) @@ -28,6 +29,25 @@ const ( FlexRoot = "v2" ) +// Extracts module name from POM path, with `rootModule` being used as the reference for +// the uppermost ancestor. The returned value should be usable with the `-pl` flag in relation +// to the POM file at `rootModule`. +func getModuleFromPomPath(pomPath string, rootModule string) (string, error) { + dir := filepath.Dir(pomPath) + allDirs := strings.Split(dir, string(os.PathSeparator)) + + i := len(allDirs) + for ; i > 0 && allDirs[i-1] != rootModule; i -= 1 { + // Empty intentionally + } + + if i == 0 { + return "", fmt.Errorf("%s is not under %s", pomPath, rootModule) + } + + return strings.Join(allDirs[i:], "/"), nil +} + // Returns a map of roots to their modules. Properties are: // Key: The root module, equivalent to one of the const values (e.g. ClassicRoot) // Value: All the submodules, sometimes nested under another parent that is also in the slice @@ -35,37 +55,26 @@ const ( // mvn x:y -f {key}/pom.xml -pl {value} // An empty value indicates no submodules. func GetModuleMapping() map[string][]string { - return map[string][]string{ - ClassicRoot: []string{}, - FlexRoot: []string{ - "bigquery-to-bigtable", - "bigquery-to-parquet", - "cdc-parent/cdc-embedded-connector", - "cdc-parent/cdc-common", - "cdc-parent", - "cdc-parent/cdc-agg", - "cdc-parent/cdc-change-applier", - "common", - "datastream-to-sql", - "datastream-to-bigquery", - "datastream-to-mongodb", - "datastream-to-postgres", - "datastream-to-spanner", - "elasticsearch-common", - "file-format-conversion", - "googlecloud-to-googlecloud", - "googlecloud-to-elasticsearch", - "hive-to-bigquery", - "kafka-to-bigquery", - "kafka-common", - "kafka-to-gcs", - "kafka-to-pubsub", - "kudu-to-bigquery", - "pubsub-binary-to-bigquery", - "pubsub-cdc-to-bigquery", - "pubsub-to-mongodb", - }, + m := make(map[string][]string) + m[ClassicRoot] = make([]string, 0) + + flexPoms, err := GetAllPomFiles(FlexRoot) + if err != nil { + // Panicking here seems reasonable, since something is deeply wrong with the filesystem + // if this fails. + panic(err) } + flexModules := make([]string, len(flexPoms)) + for i := range flexPoms { + if module, err := getModuleFromPomPath(flexPoms[i], FlexRoot); err != nil { + panic(err) + } else { + flexModules[i] = module + } + } + m[FlexRoot] = flexModules + + return m } type moduleTrieNode struct { From 90f5fa9205a6af55527f77418ec115fe802e18b6 Mon Sep 17 00:00:00 2001 From: WeiwenXu21 Date: Tue, 18 Jan 2022 13:37:06 -0500 Subject: [PATCH 036/145] Resolve issues with affected template input. Resolve wording issues. --- .github/ISSUE_TEMPLATE/bug-template.yml | 72 +++++-------------- .../feature-request-template.yml | 51 ++----------- 2 files changed, 20 insertions(+), 103 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/bug-template.yml b/.github/ISSUE_TEMPLATE/bug-template.yml index b771c825e9..dbbb7d72ba 100644 --- a/.github/ISSUE_TEMPLATE/bug-template.yml +++ b/.github/ISSUE_TEMPLATE/bug-template.yml @@ -5,55 +5,23 @@ labels: ["bug", "p2", "needs triage"] body: - type: markdown attributes: - value: | - Thank you so much for taking the time to fill out this bug report! Please note that changing the issue label from p2 to p0/p1 will *NOT* speed up triaging. **Any issue that is currently affecting your production should be directed to Google Cloud customer support.** If you cannot find the template in the dropdown menu below, we probably no longer support the template. - - type: dropdown + value: > + Thank you so much for taking the time to fill out this bug report! + + **Please note: any issue that is currently affecting your production should be directed to Google Cloud customer support.** + - type: input id: template attributes: - label: Related Template - options: - - Pubsub-to-BigQuery (classic, streaming) - - PubsubAvro-to-BigQuery (flex, steaming) - - PubsubProto-to-BigQuery (flex, streaming) - - Pubsub-to-Pubsub (classic, streaming) - - Pubsub-to-Splunk (classic, streaming) - - Pubsub-to-Avro (classic, streaming) - - Pubsub-to-Text (classic, streaming) - - Pubsub-to-MongoDB (flex, streaming) - - Pubsub-to-Elasticsearch (flex, streaming) - - Datastream-to-Spanner (flex, streaming) - - Text-to-BigQuery-stream (classic, streaming) - - Text-to-Pubsub-stream (classic, streaming) - - DLPText-to-BigQuery-stream (classic, streaming) - - CDC (flex, streaming) - - Kafka-to-BigQuery (flex, streaming) - - Datastream-to-BigQuery (flex, streaming) - - Datastream-to-Postgres (flex, streaming) - - JDBC-to-Pubsub (flex, streaming) - - Pubsub-to-JDBC (flex, streaming) - - BigQuery-to-TFRecord (classic, batch) - - BigQuery-to-Parquet (flex, batch) - - BigQuery-to-Elasticsearch (flex, batch) - - Bigtable-to-Avro (classic, batch) - - Bigtable-to-Parquet (classic, batch) - - Datastore-to-Text (classic, batch) - - Spanner-to-Avro/Spanner-Export (classic, batch) - - Spanner-to-Text (classic, batch) - - Avro-to-Bigtable (classic, batch) - - Avro-to-Spanner/Spanner-Import (classic, batch) - - Parquet-to-Bigtable (classic, batch) - - TextIO-to-BigQuery (classic, batch) - - Text-to-Datastore (classic, batch) - - Text-to-Pubsub-batch (classic, batch) - - Text-to-Spanner/Spanner-TextImport (classic, batch) - - GCS-to-Elasticsearch (flex, batch) - - JDBC-to-BigQuery (classic, batch) - - Cassandra-to-Bigtable (classic, batch) - - File-Format-Conversion (flex, util) - - Bulk-Compressor (classic, util) - - Bulk-Decompressor (classic, util) - - Firestore-Bulk-Delete (classic, util) - - Streaming-Data-Generator (flex, util) + label: Related Template(s) + description: "Which template(s) this bug is affecting?" + validations: + required: true + - type: textarea + id: what-happened + attributes: + label: What happened? + description: Please share what you expected and what happened! + placeholder: Tell us what you see! validations: required: true - type: dropdown @@ -62,6 +30,7 @@ body: label: Beam Version description: What version of beam are you using? options: + - Newer than 2.35.0 - 2.35.0 - 2.34.0 - 2.33.0 @@ -71,18 +40,9 @@ body: - Older than 2.30.0 validations: required: true - - type: textarea - id: what-happened - attributes: - label: What happened? - description: Please share what you expected and what happened! - placeholder: Tell us what you see! - validations: - required: true - type: textarea id: logs attributes: label: Relevant log output description: Please copy and paste any relevant log output. This will be automatically formatted into code, so no need for backticks. render: shell - diff --git a/.github/ISSUE_TEMPLATE/feature-request-template.yml b/.github/ISSUE_TEMPLATE/feature-request-template.yml index f52a8ea48b..d30de9b09b 100644 --- a/.github/ISSUE_TEMPLATE/feature-request-template.yml +++ b/.github/ISSUE_TEMPLATE/feature-request-template.yml @@ -6,55 +6,12 @@ body: - type: markdown attributes: value: | - Thank you so much for taking the time to fill out this feature request report! Please note that changing the issue label from p2 to p0/p1 will *NOT* speed up triaging. **Any issue that is currently affecting your production should be directed to Google Cloud customer support.** If you cannot find the template in the dropdown menu below, we probably no longer support the template. - - type: dropdown + Thank you so much for taking the time to fill out this feature request report! + - type: input id: template attributes: - label: Related Template - options: - - Request a new template - - Pubsub-to-BigQuery (classic, streaming) - - PubsubAvro-to-BigQuery (flex, steaming) - - PubsubProto-to-BigQuery (flex, streaming) - - Pubsub-to-Pubsub (classic, streaming) - - Pubsub-to-Splunk (classic, streaming) - - Pubsub-to-Avro (classic, streaming) - - Pubsub-to-Text (classic, streaming) - - Pubsub-to-MongoDB (flex, streaming) - - Pubsub-to-Elasticsearch (flex, streaming) - - Datastream-to-Spanner (flex, streaming) - - Text-to-BigQuery-stream (classic, streaming) - - Text-to-Pubsub-stream (classic, streaming) - - DLPText-to-BigQuery-stream (classic, streaming) - - CDC (flex, streaming) - - Kafka-to-BigQuery (flex, streaming) - - Datastream-to-BigQuery (flex, streaming) - - Datastream-to-Postgres (flex, streaming) - - JDBC-to-Pubsub (flex, streaming) - - Pubsub-to-JDBC (flex, streaming) - - BigQuery-to-TFRecord (classic, batch) - - BigQuery-to-Parquet (flex, batch) - - BigQuery-to-Elasticsearch (flex, batch) - - Bigtable-to-Avro (classic, batch) - - Bigtable-to-Parquet (classic, batch) - - Datastore-to-Text (classic, batch) - - Spanner-to-Avro/Spanner-Export (classic, batch) - - Spanner-to-Text (classic, batch) - - Avro-to-Bigtable (classic, batch) - - Avro-to-Spanner/Spanner-Import (classic, batch) - - Parquet-to-Bigtable (classic, batch) - - TextIO-to-BigQuery (classic, batch) - - Text-to-Datastore (classic, batch) - - Text-to-Pubsub-batch (classic, batch) - - Text-to-Spanner/Spanner-TextImport (classic, batch) - - GCS-to-Elasticsearch (flex, batch) - - JDBC-to-BigQuery (classic, batch) - - Cassandra-to-Bigtable (classic, batch) - - File-Format-Conversion (flex, util) - - Bulk-Compressor (classic, util) - - Bulk-Decompressor (classic, util) - - Firestore-Bulk-Delete (classic, util) - - Streaming-Data-Generator (flex, util) + label: Related Template(s) + description: "Please let us know which templates are you submitting this feature request for. Or if you are requesting a new template." validations: required: true - type: textarea From 213523701221079ff6f5ac8e154362e0f21d60e8 Mon Sep 17 00:00:00 2001 From: WeiwenXu21 Date: Wed, 19 Jan 2022 10:57:11 -0500 Subject: [PATCH 037/145] Fix wording issues --- .github/ISSUE_TEMPLATE/bug-template.yml | 2 +- .github/ISSUE_TEMPLATE/feature-request-template.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/bug-template.yml b/.github/ISSUE_TEMPLATE/bug-template.yml index dbbb7d72ba..aa8059f83b 100644 --- a/.github/ISSUE_TEMPLATE/bug-template.yml +++ b/.github/ISSUE_TEMPLATE/bug-template.yml @@ -13,7 +13,7 @@ body: id: template attributes: label: Related Template(s) - description: "Which template(s) this bug is affecting?" + description: "Which template(s) is this bug affecting?" validations: required: true - type: textarea diff --git a/.github/ISSUE_TEMPLATE/feature-request-template.yml b/.github/ISSUE_TEMPLATE/feature-request-template.yml index d30de9b09b..6d76a8e71b 100644 --- a/.github/ISSUE_TEMPLATE/feature-request-template.yml +++ b/.github/ISSUE_TEMPLATE/feature-request-template.yml @@ -6,7 +6,7 @@ body: - type: markdown attributes: value: | - Thank you so much for taking the time to fill out this feature request report! + Thank you so much for taking the time to fill out this feature request! - type: input id: template attributes: From 752e10349cacf7cf6f33ba8451518c40e1da2381 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 19 Jan 2022 14:36:02 -0500 Subject: [PATCH 038/145] Fix typos and rearrange to put helpers below public functions --- cicd/internal/op/maven.go | 4 +- cicd/internal/repo/files.go | 40 ++++++------- cicd/internal/repo/modules.go | 108 +++++++++++++++++----------------- 3 files changed, 76 insertions(+), 76 deletions(-) diff --git a/cicd/internal/op/maven.go b/cicd/internal/op/maven.go index 47ac88a27b..4093b6c1b1 100644 --- a/cicd/internal/op/maven.go +++ b/cicd/internal/op/maven.go @@ -20,14 +20,14 @@ import ( "fmt" ) -// Run the given Maven command on a specified POM file. Considering the input, this is equivalent to: +// Runs the given Maven command on a specified POM file. Considering the input, this is equivalent to: // mvn {cmd} -f {pomDir}/pom.xml {args...} func RunMavenOnPom(pomDir string, cmd string, args ...string) error { wa := []string{cmd, "-f", fmt.Sprintf("%s/pom.xml", pomDir)} return RunCmdAndStreamOutput("mvn", append(wa, args...)) } -// Rung the given Maven command on a specified module. Considering the input, this is equivalent to: +// Runs the given Maven command on a specified module. Considering the input, this is equivalent to: // mvn {cmd} -f {pomDir}/pom.xml -pl {module} {args...} func RunMavenOnModule(pomDir string, cmd string, module string, args ...string) error { fullArgs := []string{"-pl", module} diff --git a/cicd/internal/repo/files.go b/cicd/internal/repo/files.go index 539e8bdf8e..bd93a22e73 100644 --- a/cicd/internal/repo/files.go +++ b/cicd/internal/repo/files.go @@ -30,26 +30,6 @@ const ( RootDirName = "DataflowTemplates" ) -func getRootDir() (string, error) { - _, path, _, ok := runtime.Caller(0) - if !ok { - return "", errors.New("could not determine a starting path to get to root directory") - } - - dir := filepath.Dir(path) - allDirs := strings.Split(dir, string(os.PathSeparator)) - i := len(allDirs) - for ; i >= 0 && allDirs[i-1] != RootDirName; i -= 1 { - // Empty intentionally - } - - if i == 0 { - return "", fmt.Errorf("%s is not in the %s project somehow", dir, RootDirName) - } - - return strings.Join(allDirs[:i], string(os.PathSeparator)), nil -} - // Gets all the POM files under `dir`. `dir` is a relative path from the root of the repository. // So if the root is located at `$HOME/go/src/github.com/GoogleCloudPlatform/DataflowTemplates`, then // passing `v2` represents `$HOME/go/src/github.com/GoogleCloudPlatform/DataflowTemplates/v2`. @@ -84,3 +64,23 @@ func GetAllPomFiles(dir string) ([]string, error) { } return poms, nil } + +func getRootDir() (string, error) { + _, path, _, ok := runtime.Caller(0) + if !ok { + return "", errors.New("could not determine a starting path to get to root directory") + } + + dir := filepath.Dir(path) + allDirs := strings.Split(dir, string(os.PathSeparator)) + i := len(allDirs) + for ; i >= 0 && allDirs[i-1] != RootDirName; i -= 1 { + // Empty intentionally + } + + if i == 0 { + return "", fmt.Errorf("%s is not in the %s project somehow", dir, RootDirName) + } + + return strings.Join(allDirs[:i], string(os.PathSeparator)), nil +} diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index 1d124f3b3d..fbdc238ba9 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -29,25 +29,6 @@ const ( FlexRoot = "v2" ) -// Extracts module name from POM path, with `rootModule` being used as the reference for -// the uppermost ancestor. The returned value should be usable with the `-pl` flag in relation -// to the POM file at `rootModule`. -func getModuleFromPomPath(pomPath string, rootModule string) (string, error) { - dir := filepath.Dir(pomPath) - allDirs := strings.Split(dir, string(os.PathSeparator)) - - i := len(allDirs) - for ; i > 0 && allDirs[i-1] != rootModule; i -= 1 { - // Empty intentionally - } - - if i == 0 { - return "", fmt.Errorf("%s is not under %s", pomPath, rootModule) - } - - return strings.Join(allDirs[i:], "/"), nil -} - // Returns a map of roots to their modules. Properties are: // Key: The root module, equivalent to one of the const values (e.g. ClassicRoot) // Value: All the submodules, sometimes nested under another parent that is also in the slice @@ -77,6 +58,60 @@ func GetModuleMapping() map[string][]string { return m } +// Extracts module name from POM path, with `rootModule` being used as the reference for +// the uppermost ancestor. The returned value should be usable with the `-pl` flag in relation +// to the POM file at `rootModule`. +func getModuleFromPomPath(pomPath string, rootModule string) (string, error) { + dir := filepath.Dir(pomPath) + allDirs := strings.Split(dir, string(os.PathSeparator)) + + i := len(allDirs) + for ; i > 0 && allDirs[i-1] != rootModule; i -= 1 { + // Empty intentionally + } + + if i == 0 { + return "", fmt.Errorf("%s is not under %s", pomPath, rootModule) + } + + return strings.Join(allDirs[i:], "/"), nil +} + +// Gets all the unique modules for files whose path from the root directory is in `paths`. Example paths: +// pom.xml -> Mapped to Classic root +// v2/cdc-parent/pom.xml -> Mapped to cdc-parent under Flex Templates +// The return value has the following properties: +// Key: The path of the root module, equivalent to ClassicRoot, FlexRoot, etc. +// Value: List of modules (e.g. cdc-parent, cdc-parent/cdc-common). An empty entry represents the root itself. +func GetModulesForPaths(paths []string) map[string][]string { + if len(paths) == 0 { + return make(map[string][]string) + } + + m := make(map[string][]string) + flex := make([]string, 0) + + v2 := fmt.Sprintf("v2%s", string(os.PathSeparator)) + + for _, path := range paths { + if strings.HasPrefix(path, v2) { + flex = append(flex, strings.TrimPrefix(path, v2)) + } else { + // TODO(zhoufek): Make this more granular, especially separating .github and cicd code + // into separate "modules" + m[ClassicRoot] = make([]string, 0) + } + } + + if len(flex) > 0 { + // Even if nothing is found, we should still account for v2/ as its own module, since + // changes might be made to important files, like v2/pom.xml + m[FlexRoot] = findUniqueFlexModules(flex) + } + + return m +} + type moduleTrieNode struct { value string children map[rune]*moduleTrieNode @@ -142,38 +177,3 @@ func findUniqueFlexModules(paths []string) []string { return ret } - -// Gets all the unique modules for files whose path from the root directory is in `paths`. Example paths: -// pom.xml -> Mapped to Classic root -// v2/cdc-parent/pom.xml -> Mapped to cdc-parent under Flex Templates -// The return value has the following properties: -// Key: The path of the root module, equivalent to ClassicRoot, FlexRoot, etc. -// Value: List of modules (e.g. cdc-parent, cdc-parent/cdc-common). An empty entry represents the root itself. -func GetModulesForPaths(paths []string) map[string][]string { - if len(paths) == 0 { - return make(map[string][]string) - } - - m := make(map[string][]string) - flex := make([]string, 0) - - v2 := fmt.Sprintf("v2%s", string(os.PathSeparator)) - - for _, path := range paths { - if strings.HasPrefix(path, v2) { - flex = append(flex, strings.TrimPrefix(path, v2)) - } else { - // TODO(zhoufek): Make this more granular, especially separating .github and cicd code - // into separate "modules" - m[ClassicRoot] = make([]string, 0) - } - } - - if len(flex) > 0 { - // Even if nothing is found, we should still account for v2/ as its own module, since - // changes made be made to important files, like v2/pom.xml - m[FlexRoot] = findUniqueFlexModules(flex) - } - - return m -} From be7b5302dcc3762cb767b61bf12126b795a654af Mon Sep 17 00:00:00 2001 From: Zachary Houfek <83302349+zhoufek@users.noreply.github.com> Date: Wed, 19 Jan 2022 16:06:57 -0500 Subject: [PATCH 039/145] Create scorecard-analysis file --- .github/workflows/scorecards-analysis.yml | 55 +++++++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 .github/workflows/scorecards-analysis.yml diff --git a/.github/workflows/scorecards-analysis.yml b/.github/workflows/scorecards-analysis.yml new file mode 100644 index 0000000000..7f0628598c --- /dev/null +++ b/.github/workflows/scorecards-analysis.yml @@ -0,0 +1,55 @@ +name: Scorecards supply-chain security +on: + # Only the default branch is supported. + branch_protection_rule: + schedule: + - cron: '5 12 * * MON' + push: + branches: [ main ] + +# Declare default permissions as read only. +permissions: read-all + +jobs: + analysis: + name: Scorecards analysis + runs-on: ubuntu-latest + permissions: + # Needed to upload the results to code-scanning dashboard. + security-events: write + actions: read + contents: read + + steps: + - name: "Checkout code" + uses: actions/checkout@ec3a7ce113134d7a93b817d10a8272cb61118579 # v2.4.0 + with: + persist-credentials: false + + - name: "Run analysis" + uses: ossf/scorecard-action@e3e75cf2ffbf9364bbff86cdbdf52b23176fe492 # v1.0.1 + with: + results_file: results.sarif + results_format: sarif + # Read-only PAT token. To create it, + # follow the steps in https://github.com/ossf/scorecard-action#pat-token-creation. + repo_token: ${{ secrets.SCORECARD_READ_TOKEN }} + # Publish the results to enable scorecard badges. For more details, see + # https://github.com/ossf/scorecard-action#publishing-results. + # For private repositories, `publish_results` will automatically be set to `false`, + # regardless of the value entered here. + publish_results: true + + # Upload the results as artifacts (optional). + - name: "Upload artifact" + uses: actions/upload-artifact@82c141cc518b40d92cc801eee768e7aafc9c2fa2 # v2.3.1 + with: + name: SARIF file + path: results.sarif + retention-days: 5 + + # Upload the results to GitHub's code scanning dashboard. + - name: "Upload to code-scanning" + uses: github/codeql-action/upload-sarif@5f532563584d71fdef14ee64d17bafb34f751ce5 # v1.0.26 + with: + sarif_file: results.sarif From 462849e5fdc38e4c474a84f3c9cf131252d00994 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 20 Jan 2022 07:06:56 -0800 Subject: [PATCH 040/145] Fix JSON for Pub/Sub Proto -> BigQuery PiperOrigin-RevId: 423059350 --- .../pubsub-proto-to-bigquery-image-spec-template.json | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/v2/pubsub-binary-to-bigquery/docs/PubSubProtoToBigQuery/pubsub-proto-to-bigquery-image-spec-template.json b/v2/pubsub-binary-to-bigquery/docs/PubSubProtoToBigQuery/pubsub-proto-to-bigquery-image-spec-template.json index 1438208a33..11c85b6ab2 100644 --- a/v2/pubsub-binary-to-bigquery/docs/PubSubProtoToBigQuery/pubsub-proto-to-bigquery-image-spec-template.json +++ b/v2/pubsub-binary-to-bigquery/docs/PubSubProtoToBigQuery/pubsub-proto-to-bigquery-image-spec-template.json @@ -59,7 +59,7 @@ "^projects\\/[^\\n\\r\\/]+\\/topics\\/[^\\n\\r\\/]+$" ], "is_optional": false, - "param_type": PUBSUB_TOPIC + "param_type": "PUBSUB_TOPIC" }, { "name": "javascriptTextTransformGcsPath", @@ -69,7 +69,7 @@ "^gs:\\/\\/[^\\n\\r]+$" ], "param_type": "GCS_READ_FILE", - "is_optional": true, + "is_optional": true }, { "name": "javascriptTextTransformFunctionName", @@ -124,6 +124,6 @@ ] }, "sdk_info": { - language: JAVA + "language": "JAVA" } } \ No newline at end of file From 4df12abc2da59a1c6baee7cf79dd05a746f158a4 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 20 Jan 2022 09:19:10 -0800 Subject: [PATCH 041/145] Remove "unreleased" from Pub/Sub Proto to BigQuery This template is in the process of being rolled and no longer "unreleased" for some regions. PiperOrigin-RevId: 423084579 --- v2/pubsub-binary-to-bigquery/README.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/v2/pubsub-binary-to-bigquery/README.md b/v2/pubsub-binary-to-bigquery/README.md index 959dc9cd8e..5584824f49 100644 --- a/v2/pubsub-binary-to-bigquery/README.md +++ b/v2/pubsub-binary-to-bigquery/README.md @@ -1,10 +1,9 @@ -# Pub/Sub binary to BigQuery +# Pub/Sub Binary to BigQuery A collection of Dataflow Flex Templates to stream binary objects (Avro, Proto etc.) from Pub/Sub to BigQuery. * [Pub/Sub Avro to BigQuery](docs/PubSubAvroToBigQuery/README.md) * [Pub/Sub Protobuf to BigQuery](docs/PubSubProtoToBigQuery/README.md) - (Unreleased) Please refer to the links above for more details on the specific template. From c67c61136de84e71bba46ff24053e03137b2b12d Mon Sep 17 00:00:00 2001 From: olegsa Date: Thu, 20 Jan 2022 12:46:31 -0800 Subject: [PATCH 042/145] DLPTextToBigQueryStreaming template doesn't use headers side input The CL simplifies DLPTextToBigQueryStreaming template by getting headers when reading csv files for processing as apposed to getting them in advance and then providing them as side inputs when processing the csv files. PiperOrigin-RevId: 423135600 --- .../templates/DLPTextToBigQueryStreaming.java | 186 ++++++------------ .../DLPTextToBigQueryStreamingTest.java | 4 +- 2 files changed, 66 insertions(+), 124 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreaming.java b/src/main/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreaming.java index d08ce6b2fd..be21d62013 100644 --- a/src/main/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreaming.java +++ b/src/main/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreaming.java @@ -63,10 +63,8 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.Element; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.Watch; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; @@ -180,12 +178,11 @@ public static PipelineResult run(TokenizePipelineOptions options) { * 1) Read from the text source continuously based on default interval e.g. 30 seconds * - Setup a window for 30 secs to capture the list of files emited. * - Group by file name as key and ReadableFile as a value. - * 2) Create a side input for the window containing list of headers par file. - * 3) Output each readable file for content processing. - * 4) Split file contents based on batch size for parallel processing. - * 5) Process each split as a DLP table content request to invoke API. - * 6) Convert DLP Table Rows to BQ Table Row. - * 7) Create dynamic table and insert successfully converted records into BQ. + * 2) Output each readable file for content processing. + * 3) Split file contents based on batch size for parallel processing. + * 4) Process each split as a DLP table content request to invoke API. + * 5) Convert DLP Table Rows to BQ Table Row. + * 6) Create dynamic table and insert successfully converted records into BQ. */ PCollection>> csvFiles = @@ -214,42 +211,10 @@ public static PipelineResult run(TokenizePipelineOptions options) { .withAllowedLateness(Duration.ZERO)) .apply(GroupByKey.create()); - /* - * Side input for the window to capture list of headers for each file emited so that it can be - * used in the next transform. - */ - final PCollectionView>>> headerMap = - csvFiles - - // 2) Create a side input for the window containing list of headers par file. - .apply( - "Create Header Map", - ParDo.of( - new DoFn>, KV>>() { - - @ProcessElement - public void processElement(ProcessContext c) { - String fileKey = c.element().getKey(); - c.element() - .getValue() - .forEach( - file -> { - try (BufferedReader br = getReader(file)) { - c.output(KV.of(fileKey, getFileHeaders(br))); - - } catch (IOException e) { - LOG.error("Failed to Read File {}", e.getMessage()); - throw new RuntimeException(e); - } - }); - } - })) - .apply("View As List", View.asList()); - PCollection> bqDataMap = csvFiles - // 3) Output each readable file for content processing. + // 2) Output each readable file for content processing. .apply( "File Handler", ParDo.of( @@ -266,24 +231,22 @@ public void processElement(ProcessContext c) { } })) - // 4) Split file contents based on batch size for parallel processing. + // 3) Split file contents based on batch size for parallel processing. .apply( "Process File Contents", ParDo.of( - new CSVReader( - NestedValueProvider.of( - options.getBatchSize(), - batchSize -> { - if (batchSize != null) { - return batchSize; - } else { - return DEFAULT_BATCH_SIZE; - } - }), - headerMap)) - .withSideInputs(headerMap)) - - // 5) Create a DLP Table content request and invoke DLP API for each processsing + new CSVReader( + NestedValueProvider.of( + options.getBatchSize(), + batchSize -> { + if (batchSize != null) { + return batchSize; + } else { + return DEFAULT_BATCH_SIZE; + } + })))) + + // 4) Create a DLP Table content request and invoke DLP API for each processsing .apply( "DLP-Tokenization", ParDo.of( @@ -292,10 +255,10 @@ public void processElement(ProcessContext c) { options.getDeidentifyTemplateName(), options.getInspectTemplateName()))) - // 6) Convert DLP Table Rows to BQ Table Row + // 5) Convert DLP Table Rows to BQ Table Row .apply("Process Tokenized Data", ParDo.of(new TableRowProcessorDoFn())); - // 7) Create dynamic table and insert successfully converted records into BQ. + // 6) Create dynamic table and insert successfully converted records into BQ. bqDataMap.apply( "Write To BQ", BigQueryIO.>write() @@ -372,15 +335,9 @@ static class CSVReader extends DoFn, KV> /** This counter is used to track number of lines processed against batch size. */ private Integer lineCount; - List csvHeaders; - - public CSVReader( - ValueProvider batchSize, - PCollectionView>>> headerMap) { + public CSVReader(ValueProvider batchSize) { lineCount = 1; this.batchSize = batchSize; - this.headerMap = headerMap; - this.csvHeaders = new ArrayList<>(); } @ProcessElement @@ -389,55 +346,57 @@ public void processElement(ProcessContext c, RestrictionTracker rows = new ArrayList<>(); + Table dlpTable = null; + /** finding out EOL for this restriction so that we know the SOL */ + int endOfLine = (int) (i * batchSize.get().intValue()); + int startOfLine = (endOfLine - batchSize.get().intValue()); + + // getting the DLP table headers + Iterator csvRows = CSVFormat.DEFAULT.parse(br).iterator(); + if (!csvRows.hasNext()) { + LOG.info("File `" + c.element().getKey() + "` is empty"); + continue; + } + List dlpTableHeaders = toDlpTableHeaders(csvRows.next()); - csvHeaders = getHeaders(c.sideInput(headerMap), fileKey); - if (csvHeaders != null) { - List dlpTableHeaders = - csvHeaders.stream() - .map(header -> FieldId.newBuilder().setName(header).build()) - .collect(Collectors.toList()); - List rows = new ArrayList<>(); - Table dlpTable = null; - /** finding out EOL for this restriction so that we know the SOL */ - int endOfLine = (int) (i * batchSize.get().intValue()); - int startOfLine = (endOfLine - batchSize.get().intValue()); - /** skipping all the rows that's not part of this restriction */ - br.readLine(); - Iterator csvRows = - CSVFormat.DEFAULT.withSkipHeaderRecord().parse(br).iterator(); - for (int line = 0; line < startOfLine; line++) { - if (csvRows.hasNext()) { - csvRows.next(); - } + /** skipping all the rows that's not part of this restriction */ + for (int line = 0; line < startOfLine; line++) { + if (csvRows.hasNext()) { + csvRows.next(); } - /** looping through buffered reader and creating DLP Table Rows equals to batch */ - while (csvRows.hasNext() && lineCount <= batchSize.get()) { + } + /** looping through buffered reader and creating DLP Table Rows equals to batch */ + while (csvRows.hasNext() && lineCount <= batchSize.get()) { - CSVRecord csvRow = csvRows.next(); - rows.add(convertCsvRowToTableRow(csvRow)); - lineCount += 1; - } - /** creating DLP table and output for next transformation */ - dlpTable = Table.newBuilder().addAllHeaders(dlpTableHeaders).addAllRows(rows).build(); - c.output(KV.of(fileKey, dlpTable)); - - LOG.debug( - "Current Restriction From: {}, Current Restriction To: {}," - + " StartofLine: {}, End Of Line {}, BatchData {}", - tracker.currentRestriction().getFrom(), - tracker.currentRestriction().getTo(), - startOfLine, - endOfLine, - dlpTable.getRowsCount()); - - } else { - - throw new RuntimeException("Header Values Can't be found For file Key " + fileKey); + CSVRecord csvRow = csvRows.next(); + rows.add(convertCsvRowToTableRow(csvRow)); + lineCount += 1; } + /** creating DLP table and output for next transformation */ + dlpTable = Table.newBuilder().addAllHeaders(dlpTableHeaders).addAllRows(rows).build(); + c.output(KV.of(fileKey, dlpTable)); + + LOG.debug( + "Current Restriction From: {}, Current Restriction To: {}," + + " StartofLine: {}, End Of Line {}, BatchData {}", + tracker.currentRestriction().getFrom(), + tracker.currentRestriction().getTo(), + startOfLine, + endOfLine, + dlpTable.getRowsCount()); } } } + private static List toDlpTableHeaders(CSVRecord headerRow) { + List result = new ArrayList<>(); + for (String header : headerRow) { + result.add(FieldId.newBuilder().setName(header).build()); + } + return result; + } + /** * SDF needs to define a @GetInitialRestriction method that can create a restriction describing * the complete work for a given element. For our case this would be the total number of rows @@ -740,21 +699,6 @@ private static BufferedReader getReader(ReadableFile csvFile) { return br; } - private static List getFileHeaders(BufferedReader reader) { - List headers = new ArrayList<>(); - try { - CSVRecord csvHeader = CSVFormat.DEFAULT.parse(reader).getRecords().get(0); - csvHeader.forEach( - headerValue -> { - headers.add(headerValue); - }); - } catch (IOException e) { - LOG.error("Failed to get csv header values}", e.getMessage()); - throw new RuntimeException(e); - } - return headers; - } - private static String checkHeaderName(String name) { /** some checks to make sure BQ column names don't fail e.g. special characters */ String checkedHeader = name.replaceAll("\\s", "_"); diff --git a/src/test/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreamingTest.java b/src/test/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreamingTest.java index ad36f1f2cb..04a9fc9e35 100644 --- a/src/test/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreamingTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/DLPTextToBigQueryStreamingTest.java @@ -107,9 +107,7 @@ public void testFileIOToBigQueryStreamingE2E() throws IOException { .apply("Read File", FileIO.readMatches().withCompression(Compression.AUTO)) .apply("Add Keys", WithKeys.of(key -> "tokenization_data")) .setCoder(KvCoder.of(StringUtf8Coder.of(), ReadableFileCoder.of())) - .apply( - "Create DLP Table", - ParDo.of(new CSVReader(batchSize, headerMap)).withSideInputs(headerMap)); + .apply("Create DLP Table", ParDo.of(new CSVReader(batchSize))); PAssert.that(dlpTable) .satisfies( From 0c3aea060376f772b610468bcea6f236fe14acc5 Mon Sep 17 00:00:00 2001 From: olegsa Date: Thu, 20 Jan 2022 15:00:56 -0800 Subject: [PATCH 043/145] DataplexGigQueryToGcs handles the case when there are no tables to export The CL adds NoopTransform to the pipeline when there are no tables to export. This is needed because pipeline.run() doesn't seem to work on "empty" pipelines. PiperOrigin-RevId: 423166473 --- .../v2/templates/DataplexBigQueryToGcs.java | 3 ++ .../teleport/v2/transforms/NoopTransform.java | 42 +++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/NoopTransform.java diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index 89280d5a9a..d373783e52 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -30,6 +30,7 @@ import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn.BigQueryClientFactory; +import com.google.cloud.teleport.v2.transforms.NoopTransform; import com.google.cloud.teleport.v2.transforms.UpdateDataplexBigQueryToGcsExportMetadataTransform; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader; import com.google.cloud.teleport.v2.utils.BigQueryUtils; @@ -179,6 +180,8 @@ static Pipeline buildPipeline( if (!tables.isEmpty()) { transformPipeline(pipeline, tables, options, targetRootPath, null, null); + } else { + pipeline.apply("Nothing to export", new NoopTransform()); } return pipeline; diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/NoopTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/NoopTransform.java new file mode 100644 index 0000000000..a932721b32 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/NoopTransform.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2021 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.v2.transforms; + +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PDone; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A transform that can be used when a template detects that there is no useful work to perform. + * + *

Since the {@code pipeline.run()} is necessary to have in a template, and the run fails on a + * pipeline without transforms, this transform can be applied to a pipeline to achieve the + * do-nothing result. + */ +public class NoopTransform extends PTransform { + private static final Logger LOG = LoggerFactory.getLogger(NoopTransform.class); + + @Override + public PDone expand(PBegin input) { + LOG.info("No items to process."); + input.apply(Create.of(StringUtf8Coder.of())); + return PDone.in(input.getPipeline()); + } +} From 72c19bc5f274e5a4786ac370c02df5879bfa45c2 Mon Sep 17 00:00:00 2001 From: olegsa Date: Fri, 21 Jan 2022 14:07:08 -0800 Subject: [PATCH 044/145] DataplexFileFormatConversion handles the case when there are no files to transform The CL adds NoopTransform to the pipeline when there are no files to transform. This is needed because pipeline.run() doesn't seem to work on "empty" pipelines. PiperOrigin-RevId: 423402540 --- .../DataplexFileFormatConversion.java | 57 ++++++++++++------- 1 file changed, 37 insertions(+), 20 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java index ed35d4464a..17103ee8d6 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java @@ -25,6 +25,7 @@ import com.google.cloud.teleport.v2.transforms.AvroConverters; import com.google.cloud.teleport.v2.transforms.CsvConverters; import com.google.cloud.teleport.v2.transforms.JsonConverters; +import com.google.cloud.teleport.v2.transforms.NoopTransform; import com.google.cloud.teleport.v2.transforms.ParquetConverters; import com.google.cloud.teleport.v2.utils.Schemas; import com.google.cloud.teleport.v2.values.DataplexAssetResourceSpec; @@ -35,6 +36,7 @@ import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.function.Predicate; @@ -223,9 +225,7 @@ public static PipelineResult run( inputFilesFilter = inputFilePath -> true; break; case FAIL: - Set outputFilePaths = - getFilesFromFilePattern(addWildCard(outputBucket)) - .collect(ImmutableSet.toImmutableSet()); + Set outputFilePaths = getAllOutputFilePaths(outputBucket); inputFilesFilter = inputFilePath -> { if (outputFilePaths.contains( @@ -243,9 +243,7 @@ public static PipelineResult run( }; break; case SKIP: - outputFilePaths = - getFilesFromFilePattern(addWildCard(outputBucket)) - .collect(ImmutableSet.toImmutableSet()); + outputFilePaths = getAllOutputFilePaths(outputBucket); inputFilesFilter = inputFilePath -> !outputFilePaths.contains( @@ -266,33 +264,42 @@ public static PipelineResult run( : dataplex.getEntities( Splitter.on(',').trimResults().splitToList(options.getInputAssetOrEntitiesList())); + boolean convertingFiles = false; for (GoogleCloudDataplexV1Entity entity : entities) { ImmutableList partitions = dataplex.getPartitions(entity.getName()); if (partitions.isEmpty()) { String outputPath = outputPathProvider.outputPathFrom(entity.getDataPath(), outputBucket); - getFilesFromFilePattern(entityToFileSpec(entity)) - .filter(inputFilesFilter) - .forEach( - inputFilePath -> - pipeline.apply( - "Convert " + shortenDataplexName(entity.getName()), - new ConvertFiles(entity, inputFilePath, options, outputPath))); + Iterator inputFilePaths = + getFilesFromFilePattern(entityToFileSpec(entity)).filter(inputFilesFilter).iterator(); + convertingFiles = inputFilePaths.hasNext(); + inputFilePaths.forEachRemaining( + inputFilePath -> + pipeline.apply( + "Convert " + shortenDataplexName(entity.getName()), + new ConvertFiles(entity, inputFilePath, options, outputPath))); } else { for (GoogleCloudDataplexV1Partition partition : partitions) { String outputPath = outputPathProvider.outputPathFrom(partition.getLocation(), outputBucket); - getFilesFromFilePattern(partitionToFileSpec(partition)) - .filter(inputFilesFilter) - .forEach( - inputFilePath -> - pipeline.apply( - "Convert " + shortenDataplexName(partition.getName()), - new ConvertFiles(entity, inputFilePath, options, outputPath))); + Iterator inputFilePaths = + getFilesFromFilePattern(partitionToFileSpec(partition)) + .filter(inputFilesFilter) + .iterator(); + convertingFiles = inputFilePaths.hasNext(); + inputFilePaths.forEachRemaining( + inputFilePath -> + pipeline.apply( + "Convert " + shortenDataplexName(partition.getName()), + new ConvertFiles(entity, inputFilePath, options, outputPath))); } } } + if (!convertingFiles) { + pipeline.apply("Nothing to convert", new NoopTransform()); + } + return pipeline.run(); } @@ -325,6 +332,10 @@ private static String ensurePathEndsWithSlash(String path) { return path.endsWith("/") ? path : path + '/'; } + private static String ensurePathStartsWithFSPrefix(String path) { + return path.startsWith("gs://") || path.startsWith("/") ? path : "gs://" + path; + } + /** Example conversion: 1.json => 1.parquet; 1.abc => 1.abc.parquet. */ private static String replaceInputExtensionWithOutputExtension( String path, OutputFileFormat outputFileFormat) { @@ -352,6 +363,12 @@ private static Stream getFilesFromFilePattern(String pattern) throws IOE .map(ResourceId::toString); } + private static ImmutableSet getAllOutputFilePaths(String outputBucket) + throws IOException { + return getFilesFromFilePattern(addWildCard(ensurePathStartsWithFSPrefix(outputBucket))) + .collect(ImmutableSet.toImmutableSet()); + } + /** Convert the input file path to a new output file path. */ @FunctionalInterface interface OutputPathProvider { From 314b1b320079a35f43b24dd8d024b91a19d25ed0 Mon Sep 17 00:00:00 2001 From: anikin Date: Sun, 23 Jan 2022 19:43:06 -0800 Subject: [PATCH 045/145] Update Beam to 2.34.0 for Flex Templates. PiperOrigin-RevId: 423712020 --- v2/pom.xml | 35 +++++++++++++++++++++++++---------- 1 file changed, 25 insertions(+), 10 deletions(-) diff --git a/v2/pom.xml b/v2/pom.xml index 177022a5d7..e792c125e6 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -29,7 +29,7 @@ 1.7.4 1.0-rc6 8.7 - 2.32.0 + 2.34.0 2.1 2.10.1 30.1-jre @@ -57,15 +57,30 @@ - - - org.apache.beam - beam-sdks-java-bom - ${beam.version} - pom - import - - + + + org.apache.beam + beam-sdks-java-bom + ${beam.version} + pom + import + + + + com.google.api + gax + 2.3.0 + + + com.google.api + gax-grpc + 2.3.0 + + From 7066b9d79e6964036b997a1df5c1b436f9a61ad8 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Mon, 24 Jan 2022 12:33:06 -0800 Subject: [PATCH 046/145] Fix NullPointerException in AvroToMutation. This fixes Issue 325 PiperOrigin-RevId: 423884529 --- .../v2/transforms/BigQueryConverters.java | 6 +-- .../v2/transforms/BigQueryConvertersTest.java | 53 +++++++++++++++++++ 2 files changed, 56 insertions(+), 3 deletions(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java index af6f322313..25e380e156 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryConverters.java @@ -129,10 +129,10 @@ public Mutation apply(SchemaAndRecord record) { continue; } - String columnValue = row.get(columnName).toString(); + Object columnObj = row.get(columnName); + byte[] columnValue = columnObj == null ? null : Bytes.toBytes(columnObj.toString()); // TODO(billyjacobson): handle other types and column families - put.addColumn( - Bytes.toBytes(columnFamily()), Bytes.toBytes(columnName), Bytes.toBytes(columnValue)); + put.addColumn(Bytes.toBytes(columnFamily()), Bytes.toBytes(columnName), columnValue); } return put; } diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java index 36b5403b56..95e61a2c07 100644 --- a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/BigQueryConvertersTest.java @@ -517,4 +517,57 @@ public void testAvroToMutation() { assertThat(shortStringField).isEqualTo(Bytes.toString(CellUtil.cloneQualifier(cells.get(0)))); assertThat(shortStringFieldValue).isEqualTo(Bytes.toString(CellUtil.cloneValue(cells.get(0)))); } + + @Test + public void testAvroToMutationNullColumnValue() { + // Arrange + String rowkey = "rowkey"; + String columnFamily = "CF"; + AvroToMutation avroToMutation = + AvroToMutation.newBuilder().setColumnFamily(columnFamily).setRowkey(rowkey).build(); + + TableSchema bqSchema = + new TableSchema() + .setFields( + Arrays.asList( + new TableFieldSchema().setName(rowkey).setType("STRING"), + new TableFieldSchema().setName(shortStringField).setType("STRING"))); + + String nullableStringField = + "{" + + String.format(" \"name\" : \"%s\",", shortStringField) + + " \"type\" : [\"null\", \"string\"]," + + String.format(" \"doc\" : \"%s\"", shortStringFieldDesc) + + "}"; + Schema avroSchema = + new Schema.Parser() + .parse( + String.format( + AVRO_SCHEMA_TEMPLATE, + new StringBuilder() + .append(String.format(avroFieldTemplate, rowkey, "string", idFieldDesc)) + .append(",") + .append(nullableStringField) + .toString())); + GenericRecordBuilder builder = new GenericRecordBuilder(avroSchema); + builder.set(rowkey, idFieldValueStr); + builder.set(shortStringField, null); + Record record = builder.build(); + SchemaAndRecord inputBqData = new SchemaAndRecord(record, bqSchema); + + // Act + Mutation mutation = avroToMutation.apply(inputBqData); + + // Assert + // Assert: Rowkey is set + assertThat(Bytes.toString(mutation.getRow())).isEqualTo(idFieldValueStr); + + assertThat(mutation.getFamilyCellMap().size()).isEqualTo(1); + + // Assert: One cell was set with a value + List cells = mutation.getFamilyCellMap().get(Bytes.toBytes(columnFamily)); + assertThat(cells.size()).isEqualTo(1); + assertThat(shortStringField).isEqualTo(Bytes.toString(CellUtil.cloneQualifier(cells.get(0)))); + assertThat(CellUtil.cloneValue(cells.get(0))).isEmpty(); + } } From 924e6fd3107030611ed8c16266047fe755050048 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 1 Feb 2022 14:24:38 -0500 Subject: [PATCH 047/145] Add a composite action for setting up the environment --- .github/actions/setup-env/action.yml | 64 ++++++++++++++++++++++++++++ .github/workflows/preconditions.yml | 18 ++------ 2 files changed, 68 insertions(+), 14 deletions(-) create mode 100644 .github/actions/setup-env/action.yml diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml new file mode 100644 index 0000000000..a3a8409ca8 --- /dev/null +++ b/.github/actions/setup-env/action.yml @@ -0,0 +1,64 @@ +# Copyright 2022 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 +# +# https://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. + + +# Action used to set up the environment. This assumes that we are using +# a Linux-based VM. +# +# General steps are: +# 1. Set up Java, which templates and tests are written in +# 2. Set up Go, which our CI/CD programs are written in +# 3. Gets all the changed files +# 4. Configure options (e.g. through the `set` built-in command) + +name: 'Setup Environment' +description: 'Sets up common environment for Dataflow Templates workflows' + +inputs: + java-version: + type: string + description: 'The version of Java to install' + required: false + default: '8' + go-version: + type: string + description: 'The version of Go to install' + required: false + default: '1.17' +outputs: + changed-files: + description: 'Comma-separated list of files that were changed' + value: ${{ steps.changed-files.outputs.all_changed_files }} + +runs: + using: 'composite' + steps: + - name: Set Shell Options + run: | + set +o history + - name: Setup Java + uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 + with: + distribution: 'zulu' + java-version: ${{ inputs.java-version }} + cache: 'maven' + - name: Setup Go + uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf + with: + go-version: ${{ inputs.go-version }} + - name: Get Changed Files + id: changed-files + uses: tj-actions/changed-files@61ee456a9d0f512e7ecfdf28863634c97dae2d16 + with: + separator: ',' \ No newline at end of file diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index 224ef4db70..dfe511f50d 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -28,22 +28,12 @@ jobs: steps: - name: Checkout Code uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 - - name: Setup Java - uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 + - name: Setup Environment + id: setup-env + uses: ./.github/actions/setup-env with: - distribution: 'zulu' java-version: '11' # Spotless won't work on version 8 - cache: 'maven' - - name: Setup Go - uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf - with: - go-version: '1.17' - - name: Get Changed Files - id: changed-files - uses: tj-actions/changed-files@61ee456a9d0f512e7ecfdf28863634c97dae2d16 - with: - separator: ',' - name: Run Spotless run: | cd cicd/ && go build ./cmd/run-spotless && cd .. - ./cicd/run-spotless --changed-files="${{ steps.changed-files.outputs.all_changed_files }}" + ./cicd/run-spotless --changed-files="${{ steps.setup-env.outputs.changed-files }}" From a6ad35b5c2e02618fa7029de84309b6e5a234112 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 1 Feb 2022 14:30:15 -0500 Subject: [PATCH 048/145] Also build all the Go code --- .github/actions/setup-env/action.yml | 14 +++++++++++--- .github/workflows/preconditions.yml | 4 +--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index a3a8409ca8..514ff1e8c9 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -19,8 +19,9 @@ # General steps are: # 1. Set up Java, which templates and tests are written in # 2. Set up Go, which our CI/CD programs are written in -# 3. Gets all the changed files -# 4. Configure options (e.g. through the `set` built-in command) +# 3. Build everything under cicd/cmd +# 4. Gets all the changed files +# 5. Configure options (e.g. through the `set` built-in command) name: 'Setup Environment' description: 'Sets up common environment for Dataflow Templates workflows' @@ -61,4 +62,11 @@ runs: id: changed-files uses: tj-actions/changed-files@61ee456a9d0f512e7ecfdf28863634c97dae2d16 with: - separator: ',' \ No newline at end of file + separator: ',' + # It shouldn't take too long to build all of this, and it will at least + # make running the target program easier + - name: Build CI/CD + run: | + cd cicd/ + go build ./cmd/... + cd .. diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index dfe511f50d..b43014691b 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -34,6 +34,4 @@ jobs: with: java-version: '11' # Spotless won't work on version 8 - name: Run Spotless - run: | - cd cicd/ && go build ./cmd/run-spotless && cd .. - ./cicd/run-spotless --changed-files="${{ steps.setup-env.outputs.changed-files }}" + run: ./cicd/run-spotless --changed-files="${{ steps.setup-env.outputs.changed-files }}" From f23b120b2434ec5ec71ee18274038bef38330983 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 1 Feb 2022 14:35:18 -0500 Subject: [PATCH 049/145] Set the shell value for run commands --- .github/actions/setup-env/action.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index 514ff1e8c9..e989909740 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -46,6 +46,7 @@ runs: using: 'composite' steps: - name: Set Shell Options + shell: bash run: | set +o history - name: Setup Java @@ -66,6 +67,7 @@ runs: # It shouldn't take too long to build all of this, and it will at least # make running the target program easier - name: Build CI/CD + shell: bash run: | cd cicd/ go build ./cmd/... From 9ff4ab028f24b4c5d15c4b350ab89d7cb76c06bc Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 1 Feb 2022 15:14:42 -0500 Subject: [PATCH 050/145] Remove disabling history, which is disabled by default --- .github/actions/setup-env/action.yml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index e989909740..0af1454f15 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -21,7 +21,6 @@ # 2. Set up Go, which our CI/CD programs are written in # 3. Build everything under cicd/cmd # 4. Gets all the changed files -# 5. Configure options (e.g. through the `set` built-in command) name: 'Setup Environment' description: 'Sets up common environment for Dataflow Templates workflows' @@ -45,10 +44,6 @@ outputs: runs: using: 'composite' steps: - - name: Set Shell Options - shell: bash - run: | - set +o history - name: Setup Java uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 with: From 851addf791e87da121bd1d1cd921e2689df2fdc9 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 2 Feb 2022 14:13:19 -0500 Subject: [PATCH 051/145] Improve module handling --- cicd/cmd/run-spotless/main.go | 26 +++++++++++++------------- cicd/internal/op/run-cmd.go | 2 +- cicd/internal/repo/modules.go | 24 +++++++++++++++++------- cicd/internal/repo/modules_test.go | 18 ++++++++++++++++-- 4 files changed, 47 insertions(+), 23 deletions(-) diff --git a/cicd/cmd/run-spotless/main.go b/cicd/cmd/run-spotless/main.go index 9b39ec42bd..cda65d9435 100644 --- a/cicd/cmd/run-spotless/main.go +++ b/cicd/cmd/run-spotless/main.go @@ -44,19 +44,19 @@ func main() { modules := repo.GetModulesForPaths(s) var fullErr error - if _, ok := modules[repo.ClassicRoot]; ok { - if err := op.RunMavenOnPom(repo.ClassicRoot, SpotlessCommand); err != nil { - fullErr = err - } - } - if flex, ok := modules[repo.FlexRoot]; ok { - for _, m := range flex { - if len(m) > 0 { // Zero length represents the root, which has no Java code - if err := op.RunMavenOnModule(repo.FlexRoot, SpotlessCommand, m); err != nil && fullErr != nil { - fullErr = err - } else if err != nil { - fullErr = fmt.Errorf("%w\n%v", fullErr, err) - } + for _, root := range repo.GetAllRoots() { + if children, ok := modules[root]; ok { + var err error + if len(children) == 0 { + err = op.RunMavenOnPom(root, SpotlessCommand) + } else { + err = op.RunMavenOnModule(root, SpotlessCommand, strings.Join(children, ",")) + } + + if err != nil && fullErr == nil { + fullErr = err + } else if err != nil { + fullErr = fmt.Errorf("%w\n%v", fullErr, err) } } } diff --git a/cicd/internal/op/run-cmd.go b/cicd/internal/op/run-cmd.go index 44976e1e50..3b7428937a 100644 --- a/cicd/internal/op/run-cmd.go +++ b/cicd/internal/op/run-cmd.go @@ -36,6 +36,6 @@ func RunCmdAndStreamOutput(cmd string, args []string) error { for scanner.Scan() { fmt.Println(scanner.Text()) } - + return op.Wait() } diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index fbdc238ba9..4189e105e7 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -29,6 +29,14 @@ const ( FlexRoot = "v2" ) +// Returns all of the known roots modules. +func GetAllRoots() []string { + return []string{ + ClassicRoot, + FlexRoot, + } +} + // Returns a map of roots to their modules. Properties are: // Key: The root module, equivalent to one of the const values (e.g. ClassicRoot) // Value: All the submodules, sometimes nested under another parent that is also in the slice @@ -144,7 +152,7 @@ func flexModulesAsTrie() *moduleTrieNode { func findUniqueFlexModules(paths []string) []string { trie := flexModulesAsTrie() - modules := make(map[string]interface{}) + allModules := make([]string, 0) for _, path := range paths { curr := trie @@ -162,17 +170,19 @@ func findUniqueFlexModules(paths []string) []string { } if possible != nil { - modules[possible.value] = nil + allModules = append(allModules, possible.value) } // We don't error from not finding anything, since it could be a root-level file // that isn't part of any module. } - ret := make([]string, len(modules)) - i := 0 - for k := range modules { - ret[i] = k - i += 1 + unique := make(map[string]interface{}) + ret := make([]string, 0) + for _, m := range allModules { + if _, ok := unique[m]; !ok { + unique[m] = nil + ret = append(ret, m) + } } return ret diff --git a/cicd/internal/repo/modules_test.go b/cicd/internal/repo/modules_test.go index ec2014505d..601f6a180d 100644 --- a/cicd/internal/repo/modules_test.go +++ b/cicd/internal/repo/modules_test.go @@ -21,6 +21,14 @@ import ( "testing" ) +func TestGetAllRoots(t *testing.T) { + actual := GetAllRoots() + expected := []string{ClassicRoot, FlexRoot} + if !reflect.DeepEqual(actual, expected) { + t.Errorf("Not all roots present. Expected: %v. Got: %v", expected, actual) + } +} + func TestModuleMappingHasAllRoots(t *testing.T) { m := GetModuleMapping() if _, ok := m[ClassicRoot]; !ok { @@ -37,7 +45,13 @@ func TestGetModulesForPaths(t *testing.T) { expected map[string][]string }{ { - input: []string{"src/something", "v2/pubsub-binary-to-bigquery/avro", "src/something-else", "v2/pubsub-binary-to-bigquery/proto"}, + input: []string{ + "src/something", + "it/something", + "v2/pubsub-binary-to-bigquery/avro", + "src/something-else", + "v2/pubsub-binary-to-bigquery/proto", + }, expected: map[string][]string{ ClassicRoot: []string{}, FlexRoot: []string{"pubsub-binary-to-bigquery"}, @@ -56,7 +70,7 @@ func TestGetModulesForPaths(t *testing.T) { }, }, { - input: []string{"something", "v2/something"}, + input: []string{"something", "it/something", "v2/something"}, expected: map[string][]string{ ClassicRoot: make([]string, 0), FlexRoot: make([]string, 0), From 27bb1d71cddef94c80c875a865dfc729e42b0794 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 2 Feb 2022 14:18:21 -0500 Subject: [PATCH 052/145] Restrict permissions to read-all for precondition checks --- .github/workflows/preconditions.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/preconditions.yml index b43014691b..c3729682aa 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/preconditions.yml @@ -20,6 +20,8 @@ on: pull_request: branches: ['main'] +permissions: read-all + jobs: spotless_check: name: Spotless From e04b7c02ad642de96dc51bbec699f939ffcb8fdd Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 2 Feb 2022 12:25:42 -0800 Subject: [PATCH 053/145] Clearly note that the BigQuery -> Bigtable template is unreleased. PiperOrigin-RevId: 425949483 --- v2/bigquery-to-bigtable/README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/v2/bigquery-to-bigtable/README.md b/v2/bigquery-to-bigtable/README.md index 43331a388b..502b8865a9 100644 --- a/v2/bigquery-to-bigtable/README.md +++ b/v2/bigquery-to-bigtable/README.md @@ -3,6 +3,10 @@ The [BigQueryToBigtable](src/main/java/com/google/cloud/teleport/v2/templates/BigQueryToBigtable.java) pipeline exports data from BigQuery using a query into a Cloud Bigtable table. +NOTE: This template is currently unreleased. If you wish to use it now, you +will need to follow the steps outlined below to add it to and run it from +your own Google Cloud project. + ## Getting Started ### Requirements From 4d648301f8924535a4245cc86aabd9911d1f421d Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 3 Feb 2022 11:27:29 -0500 Subject: [PATCH 054/145] Minor improvements to GitHub Actions and Go commands --- .github/actions/setup-env/action.yml | 4 +- cicd/cmd/run-spotless/main.go | 42 ++++++++---------- cicd/internal/flags/common-flags.go | 41 ++++++++++++++++++ cicd/internal/flags/common-flags_test.go | 54 ++++++++++++++++++++++++ cicd/internal/repo/modules.go | 6 ++- cicd/internal/repo/modules_test.go | 5 ++- 6 files changed, 123 insertions(+), 29 deletions(-) create mode 100644 cicd/internal/flags/common-flags.go create mode 100644 cicd/internal/flags/common-flags_test.go diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index 0af1454f15..46ca954769 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -65,5 +65,7 @@ runs: shell: bash run: | cd cicd/ - go build ./cmd/... + for CMD in $(ls cmd); do + go build ./cmd/$CMD + done cd .. diff --git a/cicd/cmd/run-spotless/main.go b/cicd/cmd/run-spotless/main.go index cda65d9435..0329cd52bf 100644 --- a/cicd/cmd/run-spotless/main.go +++ b/cicd/cmd/run-spotless/main.go @@ -18,10 +18,10 @@ package main import ( "flag" - "fmt" "log" "strings" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/flags" "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" ) @@ -31,37 +31,31 @@ const ( ) func main() { - changed := flag.String("changed-files", "", "List of changed files as a comma-separated string") + flags.RegisterCommonFlags() flag.Parse() - if len(*changed) == 0 { - log.Print("No changed files passed. This is probably an error, but we're assuming it isn't just in case") + changed := flags.ChangedFiles() + if len(changed) == 0 { return } - log.Printf("Received changed files: %s", *changed) - s := strings.Split(*changed, ",") - modules := repo.GetModulesForPaths(s) - - var fullErr error - for _, root := range repo.GetAllRoots() { - if children, ok := modules[root]; ok { - var err error - if len(children) == 0 { - err = op.RunMavenOnPom(root, SpotlessCommand) - } else { - err = op.RunMavenOnModule(root, SpotlessCommand, strings.Join(children, ",")) - } - - if err != nil && fullErr == nil { - fullErr = err - } else if err != nil { - fullErr = fmt.Errorf("%w\n%v", fullErr, err) - } + errored := false + for root, children := range repo.GetModulesForPaths(changed) { + var err error + if len(children) == 0 { + err = op.RunMavenOnPom(root, SpotlessCommand) + } else if len(children) > 1 || children[0] != "" { + err = op.RunMavenOnModule(root, SpotlessCommand, strings.Join(children, ",")) + } else { + log.Printf("Skipping '%s' because the only files changed were not associated with a module", root) + } + + if err != nil { + errored = true } } - if fullErr != nil { + if errored { log.Fatal("There were spotless errors. Check the output from the commands.") } } diff --git a/cicd/internal/flags/common-flags.go b/cicd/internal/flags/common-flags.go new file mode 100644 index 0000000000..7a6e4053e6 --- /dev/null +++ b/cicd/internal/flags/common-flags.go @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2022 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 flags + +import ( + "flag" + "log" + "strings" +) + +// Avoid making these vars public. +var ( + changedFiles string +) + +func RegisterCommonFlags() { + flag.StringVar(&changedFiles, "changed-files", "", "List of changed files as a comma-separated string") +} + +func ChangedFiles() []string { + if len(changedFiles) == 0 { + log.Println("WARNING: No changed files were passed. This could indicate an error.") + return make([]string, 0) + } + + return strings.Split(changedFiles, ",") +} diff --git a/cicd/internal/flags/common-flags_test.go b/cicd/internal/flags/common-flags_test.go new file mode 100644 index 0000000000..200cd938c5 --- /dev/null +++ b/cicd/internal/flags/common-flags_test.go @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2022 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 flags + +import ( + "reflect" + "testing" +) + +func TestChangedFiles(t *testing.T) { + tests := []struct { + input string + expected []string + }{ + { + input: "file1,file2", + expected: []string{"file1", "file2"}, + }, + { + input: "file1", + expected: []string{"file1"}, + }, + } + + for _, test := range tests { + changedFiles = test.input + actual := ChangedFiles() + if !reflect.DeepEqual(actual, test.expected) { + t.Errorf("Returned files are not equal. Expected %v. Got %v.", test.expected, actual) + } + } +} + +func TestChangedFilesEmpty(t *testing.T) { + changedFiles = "" + actual := ChangedFiles() + if len(actual) != 0 { + t.Errorf("Expected empty slice, but got %v of len %v", actual, len(actual)) + } +} diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index 4189e105e7..d81dd0ca6e 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -171,9 +171,11 @@ func findUniqueFlexModules(paths []string) []string { if possible != nil { allModules = append(allModules, possible.value) + } else { + // Include the possibility of having no module. The receiver can decide what + // to do with it. + allModules = append(allModules, "") } - // We don't error from not finding anything, since it could be a root-level file - // that isn't part of any module. } unique := make(map[string]interface{}) diff --git a/cicd/internal/repo/modules_test.go b/cicd/internal/repo/modules_test.go index 601f6a180d..2545c5252a 100644 --- a/cicd/internal/repo/modules_test.go +++ b/cicd/internal/repo/modules_test.go @@ -51,10 +51,11 @@ func TestGetModulesForPaths(t *testing.T) { "v2/pubsub-binary-to-bigquery/avro", "src/something-else", "v2/pubsub-binary-to-bigquery/proto", + "v2/something", }, expected: map[string][]string{ ClassicRoot: []string{}, - FlexRoot: []string{"pubsub-binary-to-bigquery"}, + FlexRoot: []string{"pubsub-binary-to-bigquery", ""}, }, }, { @@ -73,7 +74,7 @@ func TestGetModulesForPaths(t *testing.T) { input: []string{"something", "it/something", "v2/something"}, expected: map[string][]string{ ClassicRoot: make([]string, 0), - FlexRoot: make([]string, 0), + FlexRoot: []string{""}, }, }, { From 51f613afb7a8210dab432c89def6a93ce88d0e56 Mon Sep 17 00:00:00 2001 From: olegsa Date: Mon, 7 Feb 2022 15:03:03 -0800 Subject: [PATCH 055/145] Dataplex Tiering template: fix fo the handling of the DATE fields. The CL updates the BigQuery to GCS transform to handle the fields with the type DATE. There is misalignment between Apache Beam and BigQuery API when translating the DATE BigQuery type to Avro type. Beam translates it as `string` while BigQuery translates it as `int` with a "logicalType" `date`. PiperOrigin-RevId: 427026994 --- .../BigQueryTableToGcsTransform.java | 34 ++++++++++++++++++- .../templates/DataplexBigQueryToGcsTest.java | 32 +++++++++-------- 2 files changed, 50 insertions(+), 16 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java index ccbc877210..7fa48feeba 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.transforms; +import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.cloud.teleport.v2.utils.BigQueryToGcsDirectoryNaming; import com.google.cloud.teleport.v2.utils.BigQueryToGcsFileNaming; import com.google.cloud.teleport.v2.utils.Schemas; @@ -22,9 +23,11 @@ import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.common.annotations.VisibleForTesting; +import java.time.LocalDate; import java.util.ArrayList; import java.util.List; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.io.AvroIO; @@ -200,7 +203,7 @@ private PCollection> transformPartition( private TypedRead getDefaultRead() { TypedRead read = - BigQueryIO.read(SchemaAndRecord::getRecord) + BigQueryIO.read(this::genericRecordWithFixedDates) .withTemplateCompatibility() // Performance hit due to validation is too big. When exporting a table with thousands // of partitions launching the job takes more than 12 minutes (Flex template timeout). @@ -215,6 +218,35 @@ private TypedRead getDefaultRead() { return testServices == null ? read : read.withTestServices(testServices); } + /** + * When Beam's BigQueryIO reads from BQ it derives the Avro schema by itself, where it maps BQ's + * `DATE` type to Avro's `string` type, so the GenericRecords outputed by the BigQueryIO contain + * `string` fields for the `DATE` columns. The Avro schema obtained from the BQ directly -- {@code + * table.getSchema()} has the `DATE` columns mapped to type Avro's `int` with logical type `date`. + * To fix this mismatch this cmethod converts the `string` dates fields to `int` with logical type + * `date` fields. + * + *

Note that for the TIMESTAMP type both Beam's BigQueryIO and BQ API map it to `long` so there + * is no mismatch. + */ + private GenericRecord genericRecordWithFixedDates(SchemaAndRecord schemaAndRecord) { + GenericRecord input = schemaAndRecord.getRecord(); + GenericRecord output = new GenericData.Record(table.getSchema()); + for (TableFieldSchema fieldSchema : schemaAndRecord.getTableSchema().getFields()) { + if ("DATE".equals(fieldSchema.getType())) { + Object value = input.get(fieldSchema.getName()); + if (!(value instanceof CharSequence)) { + throw new IllegalStateException( + "The class of input value of type DATE is " + value.getClass()); + } + output.put(fieldSchema.getName(), (int) LocalDate.parse((CharSequence) value).toEpochDay()); + } else { + output.put(fieldSchema.getName(), input.get(fieldSchema.getName())); + } + } + return output; + } + private Write getDefaultWrite() { return FileIO.write() .withNumShards(1); // Must be 1 as we can only have 1 file per partition. diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java index 83c1b055ae..c21c7ce678 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java @@ -148,6 +148,7 @@ public void setUp() throws InterruptedException, IOException { ImmutableList.of( new TableFieldSchema().setName("ts").setType("TIMESTAMP"), new TableFieldSchema().setName("s1").setType("STRING"), + new TableFieldSchema().setName("d1").setType("DATE"), new TableFieldSchema().setName("i1").setType("INTEGER"))); avroSchema = @@ -156,6 +157,7 @@ public void setUp() throws InterruptedException, IOException { "{\"type\":\"record\",\"name\":\"__root__\",\"fields\":" + "[{\"name\":\"ts\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}]}," + "{\"name\":\"s1\",\"type\":[\"null\",\"string\"]}," + + "{\"name\":\"d1\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}]}," + "{\"name\":\"i1\",\"type\":[\"null\",\"long\"]}]}"); long modTime = System.currentTimeMillis() * 1000; @@ -197,20 +199,20 @@ public void setUp() throws InterruptedException, IOException { defaultRecords = new TableRow[] { - new TableRow().set("ts", 1L).set("s1", "1001").set("i1", 2001L), - new TableRow().set("ts", 2L).set("s1", "1002").set("i1", 2002L), - new TableRow().set("ts", 3L).set("s1", "1003").set("i1", 2003L), - new TableRow().set("ts", 4L).set("s1", "1004").set("i1", null), - new TableRow().set("ts", 5L).set("s1", "1005").set("i1", 2005L) + new TableRow().set("ts", 1L).set("s1", "1001").set("d1", "1970-01-01").set("i1", 2001L), + new TableRow().set("ts", 2L).set("s1", "1002").set("d1", "1970-01-02").set("i1", 2002L), + new TableRow().set("ts", 3L).set("s1", "1003").set("d1", "1970-01-03").set("i1", 2003L), + new TableRow().set("ts", 4L).set("s1", "1004").set("d1", "1970-01-04").set("i1", null), + new TableRow().set("ts", 5L).set("s1", "1005").set("d1", "1970-01-05").set("i1", 2005L) }; defaultExpectedRecords = new String[] { - "{\"ts\": 1, \"s1\": \"1001\", \"i1\": 2001}", - "{\"ts\": 2, \"s1\": \"1002\", \"i1\": 2002}", - "{\"ts\": 3, \"s1\": \"1003\", \"i1\": 2003}", - "{\"ts\": 4, \"s1\": \"1004\", \"i1\": null}", - "{\"ts\": 5, \"s1\": \"1005\", \"i1\": 2005}" + "{\"ts\": 1, \"s1\": \"1001\", \"d1\": 0, \"i1\": 2001}", + "{\"ts\": 2, \"s1\": \"1002\", \"d1\": 1, \"i1\": 2002}", + "{\"ts\": 3, \"s1\": \"1003\", \"d1\": 2, \"i1\": 2003}", + "{\"ts\": 4, \"s1\": \"1004\", \"d1\": 3, \"i1\": null}", + "{\"ts\": 5, \"s1\": \"1005\", \"d1\": 4, \"i1\": 2005}" }; FakeDatasetService.setUp(); @@ -406,14 +408,14 @@ public void testE2E_withEnforceSamePartitionKeyEnabled_producesRenamedColumns() String[] expectedRecords1 = new String[] { - "{\"ts_pkey\": 1, \"s1\": \"1001\", \"i1\": 2001}", - "{\"ts_pkey\": 2, \"s1\": \"1002\", \"i1\": 2002}" + "{\"ts_pkey\": 1, \"s1\": \"1001\", \"d1\": 0, \"i1\": 2001}", + "{\"ts_pkey\": 2, \"s1\": \"1002\", \"d1\": 1, \"i1\": 2002}" }; String[] expectedRecords2 = new String[] { - "{\"ts_pkey\": 3, \"s1\": \"1003\", \"i1\": 2003}", - "{\"ts_pkey\": 4, \"s1\": \"1004\", \"i1\": null}", - "{\"ts_pkey\": 5, \"s1\": \"1005\", \"i1\": 2005}" + "{\"ts_pkey\": 3, \"s1\": \"1003\", \"d1\": 2, \"i1\": 2003}", + "{\"ts_pkey\": 4, \"s1\": \"1004\", \"d1\": 3, \"i1\": null}", + "{\"ts_pkey\": 5, \"s1\": \"1005\", \"d1\": 4, \"i1\": 2005}" }; PAssert.that(actualRecords1).containsInAnyOrder(expectedRecords1); From 4cf4836183e4010accfcc723e5e049f24976d767 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 15 Feb 2022 11:54:09 -0500 Subject: [PATCH 056/145] Switch to actions/cache for setting up Maven cache --- .github/actions/setup-env/action.yml | 3 +- .github/actions/setup-java-cache/action.yml | 53 +++++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) create mode 100644 .github/actions/setup-java-cache/action.yml diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index 46ca954769..2be9e78bfb 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -49,7 +49,6 @@ runs: with: distribution: 'zulu' java-version: ${{ inputs.java-version }} - cache: 'maven' - name: Setup Go uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf with: @@ -69,3 +68,5 @@ runs: go build ./cmd/$CMD done cd .. + - name: Setup Java Cache + uses: ./.github/actions/setup-java-cache diff --git a/.github/actions/setup-java-cache/action.yml b/.github/actions/setup-java-cache/action.yml new file mode 100644 index 0000000000..9910b5eeb0 --- /dev/null +++ b/.github/actions/setup-java-cache/action.yml @@ -0,0 +1,53 @@ +# Copyright 2022 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 +# +# https://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. + +# Sets up the cache for Java dependencies. If a specific key is given, then the +# key is used to resolve or else resolving the first entry that matches the prefix +# without the key. Otherwise, today's date and yetserday's date (both in UTC) will be +# tried before going to the first match without any date. + +name: 'Setup Java Cache' +description: 'Sets up the Java dependency cache.' + +inputs: + key: + type: string + description: 'Key to use for the cache entry' + required: false + default: '' + +runs: + using: 'composite' + steps: + - name: Prepare Key + shell: bash + env: + KEY: ${{ inputs.key }} + run: | + if [[ "$KEY" == "" ]]; then + echo "TODAY=$(date -u +%Y%m%d)" >> $GITHUB_ENV + echo "YESTERDAY=$(date -u --date='1 day ago' +%Y%m%d)" >> $GITHUB_ENV + else + echo "TODAY=${{ inputs.key }}" >> $GITHUB_ENV + echo "YESTERDAY=${{ inputs.key }}" >> $GITHUB_ENV + fi + - name: Setup Cache + uses: actions/cache@72d1e4fdff0ff7b1b6e86b415f2d4f5941e5c006 + with: + path: | + ~/.m2 + key: java-maven-cache-${{ env.TODAY }} + restore-keys: | + java-maven-cache-${{ env.YESTERDAY }} + java-maven-cache From 3bf7bb2d08ec2d53a27ac4c6cfacc62646376aa5 Mon Sep 17 00:00:00 2001 From: Zachary Houfek <83302349+zhoufek@users.noreply.github.com> Date: Wed, 16 Feb 2022 10:22:02 -0500 Subject: [PATCH 057/145] Create SECURITY.md --- SECURITY.md | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 SECURITY.md diff --git a/SECURITY.md b/SECURITY.md new file mode 100644 index 0000000000..4648e5e3a5 --- /dev/null +++ b/SECURITY.md @@ -0,0 +1,4 @@ +To report a security issue, please use [https://g.co/vulnz](https://g.co/vulnz). +We use g.co/vulnz for our intake, and do coordination and disclosure here on +GitHub (including using GitHub Security Advisory). The Google Security Team will +respond within 5 working days of your report on g.co/vulnz. From 7fef8f9e25949c46eb2557eb8ee27fc32602fe43 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 16 Feb 2022 10:54:08 -0500 Subject: [PATCH 058/145] Add a workflow for running checks on Go PRs --- .github/workflows/go-pr.yml | 65 +++++++++++++++++++++++++++++++++++++ 1 file changed, 65 insertions(+) create mode 100644 .github/workflows/go-pr.yml diff --git a/.github/workflows/go-pr.yml b/.github/workflows/go-pr.yml new file mode 100644 index 0000000000..8c0373ad2e --- /dev/null +++ b/.github/workflows/go-pr.yml @@ -0,0 +1,65 @@ +# Copyright 2022 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 +# +# https://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. + +# Checks to run whenever there is a Go PR. Currently, we only write Go code +# for CI/CD purposes, so the jobs tend to lighter-weight than Java PRs +# are. + +name: Go PR + +on: + pull_request: + branches: + - 'main' + paths: + - 'cicd/**' + - '.github/workflows/go-pr.yml' + +permissions: read-all + +jobs: + check_formatting: + name: Go FMT + timeout-minutes: 5 + runs-on: ubuntu-latest + steps: + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Go + uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf + with: + go-version: '1.17' + - name: Run Fmt + run: | + cd cicd + if [[ $(go fmt ./...) != "" ]]; then + echo 'There were formatting violations' + exit 1 + fi + build_and_tests: + name: Build and Test + timeout-minutes: 5 + runs-on: ubuntu-latest + steps: + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Go + uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf + with: + go-version: '1.17' + # By nature, this also makes sure that everything builds + - name: Run Tests + run: | + cd cicd + go test ./... From eabd008a937eb66e290fc742a56585a2839df984 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 16 Feb 2022 10:58:33 -0500 Subject: [PATCH 059/145] Fix a formatting violation caught by new check --- cicd/cmd/run-spotless/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cicd/cmd/run-spotless/main.go b/cicd/cmd/run-spotless/main.go index 0329cd52bf..9d36e071c0 100644 --- a/cicd/cmd/run-spotless/main.go +++ b/cicd/cmd/run-spotless/main.go @@ -49,7 +49,7 @@ func main() { } else { log.Printf("Skipping '%s' because the only files changed were not associated with a module", root) } - + if err != nil { errored = true } From 7a6ea797ee58acfe84794eaa6fc35553b2954e26 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 15 Feb 2022 15:37:42 -0500 Subject: [PATCH 060/145] Add a workflow for updating the cache --- .github/actions/setup-env/action.yml | 11 +-- .../action.yml | 29 ++++++-- .github/workflows/prepare-java-cache.yml | 74 +++++++++++++++++++ 3 files changed, 100 insertions(+), 14 deletions(-) rename .github/actions/{setup-java-cache => setup-java-env}/action.yml (67%) create mode 100644 .github/workflows/prepare-java-cache.yml diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index 2be9e78bfb..f28f3a6fdb 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -44,11 +44,6 @@ outputs: runs: using: 'composite' steps: - - name: Setup Java - uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 - with: - distribution: 'zulu' - java-version: ${{ inputs.java-version }} - name: Setup Go uses: actions/setup-go@44e221478fc6847752e5c574fc7a7b3247b00fbf with: @@ -68,5 +63,7 @@ runs: go build ./cmd/$CMD done cd .. - - name: Setup Java Cache - uses: ./.github/actions/setup-java-cache + - name: Setup Java + uses: ./.github/actions/setup-java-env + with: + java-version: ${{ inputs.java-version }} diff --git a/.github/actions/setup-java-cache/action.yml b/.github/actions/setup-java-env/action.yml similarity index 67% rename from .github/actions/setup-java-cache/action.yml rename to .github/actions/setup-java-env/action.yml index 9910b5eeb0..b2d909e439 100644 --- a/.github/actions/setup-java-cache/action.yml +++ b/.github/actions/setup-java-env/action.yml @@ -17,33 +17,48 @@ # without the key. Otherwise, today's date and yetserday's date (both in UTC) will be # tried before going to the first match without any date. -name: 'Setup Java Cache' -description: 'Sets up the Java dependency cache.' +name: 'Setup Java Environment' +description: 'Sets up the full Java environment.' inputs: - key: + cache-key: type: string description: 'Key to use for the cache entry' required: false default: '' + java-version: + type: string + description: 'The version of Java to install' + required: false + default: '8' +outputs: + cache-hit: + description: 'Whether or not there was a cache hit' + value: ${{ steps.setup-cache.outputs.cache-hit }} runs: using: 'composite' steps: + - name: Setup Java + uses: actions/setup-java@a12e082d834968c1847f782019214fadd20719f6 + with: + distribution: 'zulu' + java-version: ${{ inputs.java-version }} - name: Prepare Key shell: bash env: - KEY: ${{ inputs.key }} + KEY: ${{ inputs.cache-key }} run: | - if [[ "$KEY" == "" ]]; then + if [[ "$KEY" == "" || "$KEY" == "''" ]]; then echo "TODAY=$(date -u +%Y%m%d)" >> $GITHUB_ENV echo "YESTERDAY=$(date -u --date='1 day ago' +%Y%m%d)" >> $GITHUB_ENV else - echo "TODAY=${{ inputs.key }}" >> $GITHUB_ENV - echo "YESTERDAY=${{ inputs.key }}" >> $GITHUB_ENV + echo "TODAY=$KEY" >> $GITHUB_ENV + echo "YESTERDAY=$KEY" >> $GITHUB_ENV fi - name: Setup Cache uses: actions/cache@72d1e4fdff0ff7b1b6e86b415f2d4f5941e5c006 + id: setup-cache with: path: | ~/.m2 diff --git a/.github/workflows/prepare-java-cache.yml b/.github/workflows/prepare-java-cache.yml new file mode 100644 index 0000000000..62cc2f10b4 --- /dev/null +++ b/.github/workflows/prepare-java-cache.yml @@ -0,0 +1,74 @@ +# Copyright 2022 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 +# +# https://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. + +# Prepares the Java cache either daily on demand. Since GitHub's cache action doesn't +# allow overwriting an existing entry, this is necessary to try to keep the cached dependencies +# as up-to-date as possible, hopefully minimizing the time it takes for PR checks to +# complete. + +name: Prepare Java Cache + +on: + # At the start of each day, refresh the cache. + schedule: + - cron: '0 0 * * *' + # A way to try to force the cache forward should something cause it to get stuck. + workflow_dispatch: + inputs: + days: + type: choice + description: 'The cache entry N days ahead to claim.' + required: true + default: '1' + options: ['1', '2', '3', '4', '5', '6'] + # Run on PRs that change this workflow or our local actions just to make sure nothing breaks. + pull_request: + branches: + - 'main' + paths: + - '.github/workflows/prepare-java-chace.yml' + - '.github/actions/setup-java-env/*' + +permissions: read-all + +jobs: + cache_dependencies: + name: Cache Java Dependencies + timeout-minutes: 60 + runs-on: ubuntu-latest + steps: + - name: Set Manual Key + if: ${{ github.event_name == 'workflow_dispatch' }} + run: | + echo "$(date -u --date='${{ inputs.days }} days' +%Y%m%d)" >> $GITHUB_ENV + - name: Set Empty Key + if: ${{ github.event_name != 'workflow_dispatch' }} + run: | + echo "CACHE_KEY=''" >> $GITHUB_ENV + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Java + id: setup-java + uses: ./.github/actions/setup-java-env + with: + cache-key: ${{ env.CACHE_KEY }} + - name: Resolve Dependencies + if: ${{ steps.setup-java.outputs.cache-hit != 'true' || github.event_name == 'pull_request' }} + run: | + for DIR in $(find . -maxdepth 1 -type d); do + POMF="$DIR/pom.xml" + if [[ -f "$POMF" ]]; then + mvn -B clean install -f "$POMF" -am -amd -Dmaven.test.skip -Dcheckstyle.skip -Djib.skip -Dmdep.analyze.skip + fi + done From 1005fcb30e510be3e09af7d8b8cd6d7200f6a354 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Wed, 16 Feb 2022 11:35:24 -0800 Subject: [PATCH 061/145] Upgrade beam version to 2.36. PiperOrigin-RevId: 429100234 --- pom.xml | 5 ++++- .../TextToBigQueryStreamingTest.java | 2 +- v2/cdc-parent/cdc-common/pom.xml | 19 ++++++++++++++++++ v2/common/pom.xml | 9 +++++++++ .../io/gcp/spanner/ExposedSpannerConfig.java | 20 +++++++++++++++++++ v2/datastream-to-sql/pom.xml | 7 +++++++ .../v2/kafka/utils/KafkaCommonUtils.java | 4 ++-- .../teleport/v2/kafka/consumer/Utils.java | 4 ++-- v2/pom.xml | 2 +- 9 files changed, 65 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index 92168fec84..557dfecaa7 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ 1.0-rc6 1.8.2 8.7 - 2.34.0 + 2.36.0 1.3 2.1 1.8 @@ -658,6 +658,9 @@ org.hamcrest:hamcrest + + org.apache.beam:beam-vendor-grpc-1_43_2 + org.threeten:threetenbp diff --git a/src/test/java/com/google/cloud/teleport/templates/TextToBigQueryStreamingTest.java b/src/test/java/com/google/cloud/teleport/templates/TextToBigQueryStreamingTest.java index 45ecfc1be6..ba994ba4ac 100644 --- a/src/test/java/com/google/cloud/teleport/templates/TextToBigQueryStreamingTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/TextToBigQueryStreamingTest.java @@ -24,7 +24,7 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.teleport.values.FailsafeElement; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryInsertError; -import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.Gson; +import org.apache.beam.vendor.grpc.v1p43p2.com.google.gson.Gson; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Test; diff --git a/v2/cdc-parent/cdc-common/pom.xml b/v2/cdc-parent/cdc-common/pom.xml index c3c40dcd43..3d1e9db424 100644 --- a/v2/cdc-parent/cdc-common/pom.xml +++ b/v2/cdc-parent/cdc-common/pom.xml @@ -47,24 +47,43 @@ + com.google.api.grpc proto-google-cloud-datacatalog-v1beta1 ${data-catalog.version} compile + + + com.google.protobuf + * + + com.google.api.grpc grpc-google-cloud-datacatalog-v1beta1 ${data-catalog.version} compile + + + com.google.protobuf + * + + com.google.cloud google-cloud-datacatalog ${data-catalog.version} compile + + + com.google.protobuf + * + + org.apache.beam diff --git a/v2/common/pom.xml b/v2/common/pom.xml index 1fdc9bb1a3..3d7e167ccd 100644 --- a/v2/common/pom.xml +++ b/v2/common/pom.xml @@ -38,6 +38,7 @@ + org.apache.avro avro @@ -95,6 +96,10 @@ com.google.guava guava + + com.google.protobuf + * + @@ -132,6 +137,10 @@ com.google.guava guava + + com.google.protobuf + * + diff --git a/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java b/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java index eb5b157a6b..fce969fd2c 100644 --- a/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java +++ b/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java @@ -16,6 +16,7 @@ package org.apache.beam.sdk.io.gcp.spanner; import com.google.cloud.ServiceFactory; +import com.google.cloud.spanner.Options.RpcPriority; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import javax.annotation.Nullable; @@ -46,6 +47,8 @@ public class ExposedSpannerConfig extends SpannerConfig { private final ValueProvider maxCumulativeBackoff; + private final ValueProvider rpcPriority; + private final ServiceFactory serviceFactory; private ExposedSpannerConfig( @@ -56,6 +59,7 @@ private ExposedSpannerConfig( @Nullable ValueProvider emulatorHost, @Nullable ValueProvider commitDeadline, @Nullable ValueProvider maxCumulativeBackoff, + @Nullable ValueProvider rpcPriority, @Nullable ServiceFactory serviceFactory) { this.projectId = projectId; this.instanceId = instanceId; @@ -64,6 +68,7 @@ private ExposedSpannerConfig( this.emulatorHost = emulatorHost; this.commitDeadline = commitDeadline; this.maxCumulativeBackoff = maxCumulativeBackoff; + this.rpcPriority = rpcPriority; this.serviceFactory = serviceFactory; } @@ -109,6 +114,12 @@ public ValueProvider getMaxCumulativeBackoff() { return maxCumulativeBackoff; } + @Nullable + @Override + public ValueProvider getRpcPriority() { + return rpcPriority; + } + @Nullable @VisibleForTesting @Override @@ -226,6 +237,7 @@ static final class Builder extends SpannerConfig.Builder { private ValueProvider emulatorHost; private ValueProvider commitDeadline; private ValueProvider maxCumulativeBackoff; + private ValueProvider rpcPriority; private ServiceFactory serviceFactory; Builder() {} @@ -238,6 +250,7 @@ private Builder(SpannerConfig source) { this.emulatorHost = source.getEmulatorHost(); this.commitDeadline = source.getCommitDeadline(); this.maxCumulativeBackoff = source.getMaxCumulativeBackoff(); + this.rpcPriority = source.getRpcPriority(); this.serviceFactory = source.getServiceFactory(); } @@ -283,6 +296,12 @@ SpannerConfig.Builder setMaxCumulativeBackoff(ValueProvider maxCumulat return this; } + @Override + SpannerConfig.Builder setRpcPriority(ValueProvider rpcPriority) { + this.rpcPriority = rpcPriority; + return this; + } + @Override ExposedSpannerConfig.Builder setServiceFactory( ServiceFactory serviceFactory) { @@ -300,6 +319,7 @@ public ExposedSpannerConfig build() { this.emulatorHost, this.commitDeadline, this.maxCumulativeBackoff, + this.rpcPriority, this.serviceFactory); } } diff --git a/v2/datastream-to-sql/pom.xml b/v2/datastream-to-sql/pom.xml index ca2935a19f..e6657c418f 100644 --- a/v2/datastream-to-sql/pom.xml +++ b/v2/datastream-to-sql/pom.xml @@ -37,6 +37,7 @@ + com.google.cloud.teleport.v2 common @@ -52,6 +53,12 @@ mysql mysql-connector-java 8.0.13 + + + com.google.protobuf + * + + diff --git a/v2/kafka-common/src/main/java/com/google/cloud/teleport/v2/kafka/utils/KafkaCommonUtils.java b/v2/kafka-common/src/main/java/com/google/cloud/teleport/v2/kafka/utils/KafkaCommonUtils.java index 7f06e8f8d2..142b945993 100644 --- a/v2/kafka-common/src/main/java/com/google/cloud/teleport/v2/kafka/utils/KafkaCommonUtils.java +++ b/v2/kafka-common/src/main/java/com/google/cloud/teleport/v2/kafka/utils/KafkaCommonUtils.java @@ -18,8 +18,8 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; -import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.JsonObject; -import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.JsonParser; +import org.apache.beam.vendor.grpc.v1p43p2.com.google.gson.JsonObject; +import org.apache.beam.vendor.grpc.v1p43p2.com.google.gson.JsonParser; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; import org.apache.http.client.methods.HttpGet; diff --git a/v2/kafka-to-pubsub/src/main/java/com/google/cloud/teleport/v2/kafka/consumer/Utils.java b/v2/kafka-to-pubsub/src/main/java/com/google/cloud/teleport/v2/kafka/consumer/Utils.java index e226ee3225..6a636c8130 100644 --- a/v2/kafka-to-pubsub/src/main/java/com/google/cloud/teleport/v2/kafka/consumer/Utils.java +++ b/v2/kafka-to-pubsub/src/main/java/com/google/cloud/teleport/v2/kafka/consumer/Utils.java @@ -24,8 +24,8 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; -import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.JsonObject; -import org.apache.beam.vendor.grpc.v1p36p0.com.google.gson.JsonParser; +import org.apache.beam.vendor.grpc.v1p43p2.com.google.gson.JsonObject; +import org.apache.beam.vendor.grpc.v1p43p2.com.google.gson.JsonParser; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; import org.apache.http.client.methods.HttpGet; diff --git a/v2/pom.xml b/v2/pom.xml index e792c125e6..54b3cc7bd1 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -29,7 +29,7 @@ 1.7.4 1.0-rc6 8.7 - 2.34.0 + 2.36.0 2.1 2.10.1 30.1-jre From 1393583de32533b386ce092bc7b15ba9c9391493 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Wed, 16 Feb 2022 13:10:42 -0800 Subject: [PATCH 062/145] Move common code to separate files. Moving common code to a separate files so it can be reused in different templates. PiperOrigin-RevId: 429122854 --- .../options/DataplexBigQueryToGcsOptions.java | 12 ++--- .../options/DataplexJdbcIngestionOptions.java | 6 +-- .../v2/templates/DataplexBigQueryToGcs.java | 27 +--------- .../DataplexFileFormatConversion.java | 45 ++++++----------- .../BigQueryTableToGcsTransform.java | 46 +++-------------- .../GenericRecordsToGcsPartitioned.java | 24 ++------- .../v2/utils/DataplexBigQueryToGcsFilter.java | 14 +----- .../cloud/teleport/v2/utils/FileFormat.java | 37 ++++++++++++++ .../cloud/teleport/v2/utils/StorageUtils.java | 34 +++++++++++++ .../teleport/v2/utils/WriteDisposition.java | 47 ++++++++++++++++++ .../templates/DataplexBigQueryToGcsTest.java | 38 +++++--------- .../DataplexFileFormatConversionTest.java | 20 ++++---- .../GenericRecordsToGcsPartitionedTest.java | 12 ++--- .../DataplexBigQueryToGcsFilterTest.java | 18 +++---- .../teleport/v2/utils/StorageUtilsTest.java | 49 +++++++++++++++++++ 15 files changed, 241 insertions(+), 188 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/FileFormat.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/WriteDisposition.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/StorageUtilsTest.java diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java index b9968edc78..ec0b167fee 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexBigQueryToGcsOptions.java @@ -15,10 +15,10 @@ */ package com.google.cloud.teleport.v2.options; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.transforms.DeleteBigQueryDataFn; import com.google.cloud.teleport.v2.transforms.UpdateDataplexBigQueryToGcsExportMetadataTransform; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionOptions; import com.google.cloud.teleport.v2.values.DataplexCompression; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.Default; @@ -94,9 +94,9 @@ public interface DataplexBigQueryToGcsOptions @Description("Output file format in GCS. Format: PARQUET, AVRO, or ORC. Default: PARQUET.") @Default.Enum("PARQUET") @Required - FileFormat getFileFormat(); + FileFormatOptions getFileFormat(); - void setFileFormat(FileFormat fileFormat); + void setFileFormat(FileFormatOptions fileFormat); @Description( "Output file compression. Format: UNCOMPRESSED, SNAPPY, GZIP, or BZIP2. Default:" @@ -116,9 +116,9 @@ public interface DataplexBigQueryToGcsOptions "Specifies the action that occurs if destination file already exists. Format: OVERWRITE," + " FAIL, SKIP. Default: SKIP.") @Default.Enum("SKIP") - WriteDisposition getWriteDisposition(); + WriteDispositionOptions getWriteDisposition(); - void setWriteDisposition(WriteDisposition writeDisposition); + void setWriteDisposition(WriteDispositionOptions writeDisposition); @Description( "Due to a BigQuery limitation, it's not possible to have a partitioned external table with" diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java index 1eed8fa069..b3d64449c4 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java @@ -15,8 +15,8 @@ */ package com.google.cloud.teleport.v2.options; -import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned.OutputFileFormat; import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned.PartitioningSchema; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.options.Default; @@ -134,7 +134,7 @@ public interface DataplexJdbcIngestionOptions extends GcpOptions, PipelineOption @Description("Output file format in GCS. Format: PARQUET, AVRO, or ORC. Default: PARQUET.") @Default.Enum("PARQUET") - OutputFileFormat getFileFormat(); + FileFormatOptions getFileFormat(); - void setFileFormat(OutputFileFormat fileFormat); + void setFileFormat(FileFormatOptions fileFormat); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java index d373783e52..fa1b8e5d47 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcs.java @@ -17,7 +17,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import static java.util.stream.Collectors.toList; import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1Asset; import com.google.cloud.bigquery.BigQuery; @@ -35,6 +34,7 @@ import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader; import com.google.cloud.teleport.v2.utils.BigQueryUtils; import com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter; +import com.google.cloud.teleport.v2.utils.StorageUtils; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexAssetResourceSpec; @@ -44,10 +44,6 @@ import java.util.List; import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; -import org.apache.beam.sdk.io.fs.MatchResult; -import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Flatten; @@ -61,7 +57,6 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -170,7 +165,7 @@ static Pipeline buildPipeline( throws ExecutionException, InterruptedException { Pipeline pipeline = Pipeline.create(options); - List existingTargetFiles = getFilesInDirectory(targetRootPath); + List existingTargetFiles = StorageUtils.getFilesInDirectory(targetRootPath); LOG.info("Loading BigQuery metadata..."); List tables = @@ -264,22 +259,4 @@ private static String resolveAsset( LOG.info("Resolved resource name: {}", resourceName); return resourceName; } - - @VisibleForTesting - static List getFilesInDirectory(String path) { - try { - String pathPrefix = path + "/"; - MatchResult result = FileSystems.match(pathPrefix + "**", EmptyMatchTreatment.ALLOW); - List fileNames = - result.metadata().stream() - .map(MatchResult.Metadata::resourceId) - .map(ResourceId::toString) - .map(s -> StringUtils.removeStart(s, pathPrefix)) - .collect(toList()); - LOG.info("{} file(s) found in directory {}", fileNames.size(), path); - return fileNames; - } catch (Exception e) { - throw new RuntimeException(e); - } - } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java index 17103ee8d6..547f12fe79 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversion.java @@ -27,7 +27,10 @@ import com.google.cloud.teleport.v2.transforms.JsonConverters; import com.google.cloud.teleport.v2.transforms.NoopTransform; import com.google.cloud.teleport.v2.transforms.ParquetConverters; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.utils.Schemas; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionOptions; import com.google.cloud.teleport.v2.values.DataplexAssetResourceSpec; import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.common.base.Splitter; @@ -90,9 +93,9 @@ public interface FileFormatConversionOptions @Description("Output file format. Format: PARQUET, AVRO, or ORC. Default: none.") @Required - OutputFileFormat getOutputFileFormat(); + FileFormatOptions getOutputFileFormat(); - void setOutputFileFormat(OutputFileFormat outputFileFormat); + void setOutputFileFormat(FileFormatOptions outputFileFormat); @Description( "Output file compression. Format: UNCOMPRESSED, SNAPPY, GZIP, or BZIP2. Default:" @@ -112,9 +115,9 @@ public interface FileFormatConversionOptions "Specifies the behaviour if output files already exist. Format: OVERWRITE," + " FAIL, SKIP. Default: OVERWRITE.") @Default.Enum("SKIP") - ExistingOutputFilesBehaviour getWriteDisposition(); + WriteDispositionOptions getWriteDisposition(); - void setWriteDisposition(ExistingOutputFilesBehaviour value); + void setWriteDisposition(WriteDispositionOptions value); } /** Supported input file formats. */ @@ -125,26 +128,6 @@ public enum InputFileFormat { AVRO } - /** Supported output file formats. */ - public enum OutputFileFormat { - PARQUET(".parquet"), - AVRO(".avro"), - ORC(".orc"); - - private final String fileSuffix; - - OutputFileFormat(String fileSuffix) { - this.fileSuffix = fileSuffix; - } - } - - /** The enum that defines how to handle existing output files. */ - public enum ExistingOutputFilesBehaviour { - OVERWRITE, - SKIP, - FAIL - } - private static final ImmutableSet EXPECTED_INPUT_FILES_EXTENSIONS = ImmutableSet.of(".csv", ".json", ".parquet", ".avro"); @@ -234,7 +217,7 @@ public static PipelineResult run( inputFilePath, outputBucket, options.getOutputFileFormat()))) { - throw new RuntimeException( + throw new WriteDispositionException( String.format( "The file %s already exists in the output asset bucket: %s", inputFilePath, outputBucket)); @@ -254,7 +237,7 @@ public static PipelineResult run( options.getOutputFileFormat())); break; default: - throw new IllegalArgumentException( + throw new UnsupportedOperationException( "Unsupported existing file behaviour: " + options.getWriteDisposition()); } @@ -338,13 +321,13 @@ private static String ensurePathStartsWithFSPrefix(String path) { /** Example conversion: 1.json => 1.parquet; 1.abc => 1.abc.parquet. */ private static String replaceInputExtensionWithOutputExtension( - String path, OutputFileFormat outputFileFormat) { + String path, FileFormatOptions outputFileFormat) { String inputFileExtension = path.substring(path.lastIndexOf('.')); if (EXPECTED_INPUT_FILES_EXTENSIONS.contains(inputFileExtension)) { return path.substring(0, path.length() - inputFileExtension.length()) - + outputFileFormat.fileSuffix; + + outputFileFormat.getFileSuffix(); } else { - return path + outputFileFormat.fileSuffix; + return path + outputFileFormat.getFileSuffix(); } } @@ -352,7 +335,7 @@ private static String inputFilePathToOutputFilePath( OutputPathProvider outputPathProvider, String inputFilePath, String outputBucket, - OutputFileFormat outputFileFormat) { + FileFormatOptions outputFileFormat) { return replaceInputExtensionWithOutputExtension( outputPathProvider.outputPathFrom(inputFilePath, outputBucket), outputFileFormat); } @@ -384,7 +367,7 @@ private static class ConvertFiles extends PTransform { private final GoogleCloudDataplexV1Entity entity; private final String inputFilePath; - private final OutputFileFormat outputFileFormat; + private final FileFormatOptions outputFileFormat; private final DataplexCompression outputFileCompression; private final String outputPath; diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java index 7fa48feeba..711aa2d004 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java @@ -18,6 +18,7 @@ import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.cloud.teleport.v2.utils.BigQueryToGcsDirectoryNaming; import com.google.cloud.teleport.v2.utils.BigQueryToGcsFileNaming; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.utils.Schemas; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; @@ -58,7 +59,7 @@ * *

If the table is not partitioned, the partition key in the output will be @{@code null}. * - *

See {@link FileFormat} for the list of supported output formats. + *

See {@link FileFormatOptions} for the list of supported output formats. */ public class BigQueryTableToGcsTransform extends PTransform>> { @@ -66,7 +67,7 @@ public class BigQueryTableToGcsTransform private static final String PARTITION_COLUMN_RENAME_SUFFIX = "_pkey"; private final BigQueryTable table; - private final FileFormat outputFileFormat; + private final FileFormatOptions outputFileFormat; private final DataplexCompression outputFileCompression; private final String targetRootPath; private final boolean enforceSamePartitionKey; @@ -75,7 +76,7 @@ public class BigQueryTableToGcsTransform public BigQueryTableToGcsTransform( BigQueryTable table, String targetRootPath, - FileFormat outputFileFormat, + FileFormatOptions outputFileFormat, DataplexCompression outputFileCompression, boolean enforceSamePartitionKey) { this.table = table; @@ -147,7 +148,8 @@ private PCollection> transformTable( getDefaultWrite() .via(sink) .withNaming( - new BigQueryToGcsFileNaming(outputFileFormat.fileSuffix, table.getTableName())) + new BigQueryToGcsFileNaming( + outputFileFormat.getFileSuffix(), table.getTableName())) .to(targetPath)) .getPerDestinationOutputFilenames() .apply( @@ -189,7 +191,7 @@ private PCollection> transformPartition( .via(sink) .withNaming( new BigQueryToGcsFileNaming( - outputFileFormat.fileSuffix, + outputFileFormat.getFileSuffix(), table.getTableName(), partition.getPartitionName())) .to(targetPath)) @@ -265,38 +267,4 @@ public BigQueryTableToGcsTransform withTestServices(BigQueryServices services) { this.testServices = services; return this; } - - /** Possible output file formats supported by {@link BigQueryTableToGcsTransform}. */ - public enum FileFormat { - PARQUET(".parquet"), - AVRO(".avro"), - ORC(".orc"); - - private final String fileSuffix; - - FileFormat(String fileSuffix) { - this.fileSuffix = fileSuffix; - } - - public String getFileSuffix() { - return fileSuffix; - } - } - - /** Possible write disposition supported by {@link BigQueryTableToGcsTransform}. */ - public enum WriteDisposition { - OVERWRITE("OVERWRITE"), - SKIP("SKIP"), - FAIL("FAIL"); - - private final String writeDisposition; - - WriteDisposition(String writeDisposition) { - this.writeDisposition = writeDisposition; - } - - public String getWriteDisposition() { - return writeDisposition; - } - } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java index 18d49036f7..af36f7f4c9 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java @@ -18,6 +18,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import com.google.cloud.teleport.v2.io.AvroSinkWithJodaDatesConversion; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.utils.SchemaUtils; import com.google.cloud.teleport.v2.values.PartitionMetadata; import com.google.common.collect.ImmutableList; @@ -112,14 +113,14 @@ private List> toPartition(ZonedDateTime dateTime) { @Nullable private final PartitioningSchema partitioningSchema; - private final OutputFileFormat outputFileFormat; + private final FileFormatOptions outputFileFormat; public GenericRecordsToGcsPartitioned( String gcsPath, String serializedAvroSchema, @Nullable String partitionColumnName, @Nullable PartitioningSchema partitioningSchema, - OutputFileFormat outputFileFormat) { + FileFormatOptions outputFileFormat) { this.gcsPath = gcsPath; this.serializedAvroSchema = serializedAvroSchema; this.partitionColumnName = partitionColumnName; @@ -151,7 +152,7 @@ public PCollection expand(PCollection input) { .apply( "Write to Storage with No Partition", FileIO.write() - .withSuffix(outputFileFormat.fileSuffix) + .withSuffix(outputFileFormat.getFileSuffix()) .via(sink) .to(gcsPath)) // Dummy conversion to Dataplex partition metadata @@ -288,21 +289,4 @@ private static String partitionToPath(List> partition) { private static String withoutFileName(String gcsPath) { return gcsPath.substring(0, gcsPath.lastIndexOf('/')); } - - /** Supported output file formats. */ - public enum OutputFileFormat { - PARQUET(".parquet"), - AVRO(".avro"), - ORC(".orc"); - - private final String fileSuffix; - - OutputFileFormat(String fileSuffix) { - this.fileSuffix = fileSuffix; - } - - public String getFileSuffix() { - return fileSuffix; - } - } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java index ed1ad87c79..dc84ef815f 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilter.java @@ -18,6 +18,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionException; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.common.base.Splitter; @@ -78,7 +79,7 @@ public DataplexBigQueryToGcsFilter( this.includePartitions = null; } - this.writeDisposition = options.getWriteDisposition().getWriteDisposition(); + this.writeDisposition = options.getWriteDisposition().getWriteDispositionOption(); this.fileSuffix = options.getFileFormat().getFileSuffix(); this.existingTargetFiles = existingTargetFiles; this.directoryNaming = new BigQueryToGcsDirectoryNaming(options.getEnforceSamePartitionKey()); @@ -196,15 +197,4 @@ public String partitionTargetFileName( .getFilename(null, null, 0, 0, null); return dirName + "/" + fileName; } - - /** - * Thrown if {@link - * com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition - * WriteDisposition} is set to {@code FAIL} and a target file exists. - */ - public static class WriteDispositionException extends RuntimeException { - public WriteDispositionException(String message) { - super(message); - } - } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/FileFormat.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/FileFormat.java new file mode 100644 index 0000000000..96f8358bc8 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/FileFormat.java @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +/** Exposes FileFormatOptions. */ +public class FileFormat { + + /** Possible output file formats supported by Dataplex Templates. */ + public enum FileFormatOptions { + PARQUET(".parquet"), + AVRO(".avro"), + ORC(".orc"); + + private final String fileSuffix; + + FileFormatOptions(String fileSuffix) { + this.fileSuffix = fileSuffix; + } + + public String getFileSuffix() { + return fileSuffix; + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/StorageUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/StorageUtils.java index 221736e11a..b6e83bfead 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/StorageUtils.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/StorageUtils.java @@ -15,11 +15,22 @@ */ package com.google.cloud.teleport.v2.utils; +import static java.util.stream.Collectors.toList; + +import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** A set of helper functions and classes for Cloud Storage. */ public class StorageUtils { + private static final Logger LOG = LoggerFactory.getLogger(StorageUtils.class); private static final Pattern BUCKET_URN_SPEC = Pattern.compile("^projects/.*/buckets/(.+)$"); /** @@ -39,4 +50,27 @@ public static String parseBucketUrn(String bucketUrn) { } return match.group(1); } + + /** + * Returns a list of all the files in a particular path. + * + * @param path input path + * @return list of all the files in the path + */ + public static List getFilesInDirectory(String path) { + try { + String pathPrefix = path + "/"; + MatchResult result = FileSystems.match(pathPrefix + "**", EmptyMatchTreatment.ALLOW); + List fileNames = + result.metadata().stream() + .map(MatchResult.Metadata::resourceId) + .map(ResourceId::toString) + .map(s -> StringUtils.removeStart(s, pathPrefix)) + .collect(toList()); + LOG.info("{} file(s) found in directory {}", fileNames.size(), path); + return fileNames; + } catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/WriteDisposition.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/WriteDisposition.java new file mode 100644 index 0000000000..d1956ff4da --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/WriteDisposition.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +/** Exposes WriteDispositionOptions and WriteDispositionException. */ +public class WriteDisposition { + + /** Provides the possible WriteDispositionOptions when writing to GCS and target file exists. */ + public enum WriteDispositionOptions { + OVERWRITE("OVERWRITE"), + SKIP("SKIP"), + FAIL("FAIL"); + + private final String writeDispositionOption; + + WriteDispositionOptions(String writeDispositionOption) { + this.writeDispositionOption = writeDispositionOption; + } + + public String getWriteDispositionOption() { + return writeDispositionOption; + } + } + + /** + * Thrown if {@link com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionOptions + * WriteDispositionOptions} is set to {@code FAIL} and a target file exists. + */ + public static class WriteDispositionException extends RuntimeException { + public WriteDispositionException(String message) { + super(message); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java index c21c7ce678..77bf6ee0de 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java @@ -47,11 +47,11 @@ import com.google.cloud.bigquery.storage.v1beta1.BigQueryStorageClient; import com.google.cloud.bigquery.storage.v1beta1.Storage.ReadSession; import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader; -import com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.utils.Schemas; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionOptions; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import com.google.cloud.teleport.v2.values.DataplexCompression; @@ -322,7 +322,7 @@ public void testE2E_mainPathWithAllStepsEnabled() throws Exception { @Category(NeedsRunner.class) public void testE2E_withAvroFileFormatAndGzipCompression_producesAvroFiles() throws Exception { insertTableData("unpartitioned_table", defaultRecords); - options.setFileFormat(FileFormat.AVRO); + options.setFileFormat(FileFormatOptions.AVRO); options.setFileCompression(DataplexCompression.GZIP); runTransform("unpartitioned_table"); @@ -367,7 +367,7 @@ public void testE2E_withDeleteSourceDataDisabled_doesntTruncateData() throws Exc @Category(NeedsRunner.class) public void testE2E_withEnforceSamePartitionKeyEnabled_producesRenamedColumns() throws Exception { options.setEnforceSamePartitionKey(true); - options.setFileFormat(FileFormat.AVRO); + options.setFileFormat(FileFormatOptions.AVRO); insertPartitionData("partitioned_table", "p1", Arrays.copyOfRange(defaultRecords, 0, 2)); insertPartitionData("partitioned_table", "p2", Arrays.copyOfRange(defaultRecords, 2, 5)); @@ -440,8 +440,8 @@ public void testE2E_withEnforceSamePartitionKeyEnabled_producesRenamedColumns() @Test public void testE2E_withTargetStrategyFail_throwsException() throws Exception { - options.setFileFormat(FileFormat.PARQUET); - options.setWriteDisposition(WriteDisposition.FAIL); + options.setFileFormat(FileFormatOptions.PARQUET); + options.setWriteDisposition(WriteDispositionOptions.FAIL); writeOutputFile("unpartitioned_table", "output-unpartitioned_table.parquet", "Test data"); @@ -472,8 +472,8 @@ public void testE2E_withTargetStrategyFail_throwsException() throws Exception { @Test public void testE2E_withTargetStrategyFail_andEnforceSamePartitionKeyEnabled_throwsException() throws Exception { - options.setFileFormat(FileFormat.PARQUET); - options.setWriteDisposition(WriteDisposition.FAIL); + options.setFileFormat(FileFormatOptions.PARQUET); + options.setWriteDisposition(WriteDispositionOptions.FAIL); options.setEnforceSamePartitionKey(true); writeOutputFile("partitioned_table/ts=p2", "output-partitioned_table-p2.parquet", "Test data"); @@ -511,8 +511,8 @@ public void testE2E_withTargetStrategyFail_andEnforceSamePartitionKeyEnabled_thr @Test @Category(NeedsRunner.class) public void testE2E_withTargetStrategySkip_skipsTable() throws Exception { - options.setFileFormat(FileFormat.PARQUET); - options.setWriteDisposition(WriteDisposition.SKIP); + options.setFileFormat(FileFormatOptions.PARQUET); + options.setWriteDisposition(WriteDispositionOptions.SKIP); File outputFile = writeOutputFile("unpartitioned_table", "output-unpartitioned_table.parquet", "Test data"); @@ -526,22 +526,6 @@ public void testE2E_withTargetStrategySkip_skipsTable() throws Exception { assertThat(readFirstLine(outputFile)).isEqualTo("Test data"); } - @Test - public void testGetFilesInDirectory_withValidPath_returnsPathsOfFilesInDirectory() - throws Exception { - File outputDir1 = tmpDir.newFolder("out", "unpartitioned_table"); - File outputFile1 = - new File(outputDir1.getAbsolutePath() + "/" + "output-unpartitioned_table.parquet"); - outputFile1.createNewFile(); - File outputDir2 = tmpDir.newFolder("out", "partitioned_table", "p2_pid=partition"); - File outputFile2 = - new File(outputDir2.getAbsolutePath() + "/" + "output-partitioned_table-partition.parquet"); - outputFile2.createNewFile(); - - List files = DataplexBigQueryToGcs.getFilesInDirectory(outDir.getAbsolutePath()); - assertThat(files.size()).isEqualTo(2); - } - private String readFirstLine(File outputFile) throws FileNotFoundException { Scanner fileReader = new Scanner(outputFile); String result = fileReader.nextLine(); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java index feaa2a4fb9..5308fcb28e 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java @@ -26,12 +26,12 @@ import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1SchemaSchemaField; import com.google.api.services.dataplex.v1.model.GoogleCloudDataplexV1StorageFormat; import com.google.cloud.teleport.v2.clients.DataplexClient; -import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.ExistingOutputFilesBehaviour; import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.FileFormatConversionOptions; import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.InputFileFormat; -import com.google.cloud.teleport.v2.templates.DataplexFileFormatConversion.OutputFileFormat; import com.google.cloud.teleport.v2.transforms.AvroConverters; import com.google.cloud.teleport.v2.transforms.ParquetConverters; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionOptions; import com.google.cloud.teleport.v2.values.DataplexAssetResourceSpec; import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.cloud.teleport.v2.values.EntityMetadata.StorageSystem; @@ -174,7 +174,7 @@ public void testEntityWithPartitionsCsvToAvroE2E() throws IOException { FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(entity1.getName()); - options.setOutputFileFormat(OutputFileFormat.AVRO); + options.setOutputFileFormat(FileFormatOptions.AVRO); options.setOutputAsset(outputAsset.getName()); DataplexFileFormatConversion.run( @@ -205,7 +205,7 @@ public void testAssetWithEntityJsonToGzippedParquetE2E() throws IOException { FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(asset2.getName()); - options.setOutputFileFormat(OutputFileFormat.PARQUET); + options.setOutputFileFormat(FileFormatOptions.PARQUET); options.setOutputAsset(outputAsset.getName()); options.setOutputFileCompression(DataplexCompression.GZIP); @@ -238,7 +238,7 @@ public void testAssetWithEntityAvroToParquetE2E() throws IOException { FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(entity3.getName()); - options.setOutputFileFormat(OutputFileFormat.PARQUET); + options.setOutputFileFormat(FileFormatOptions.PARQUET); options.setOutputAsset(outputAsset.getName()); DataplexFileFormatConversion.run( @@ -270,7 +270,7 @@ public void testAssetWithEntityParquetToAvroE2E() throws IOException { FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(entity4.getName()); - options.setOutputFileFormat(OutputFileFormat.AVRO); + options.setOutputFileFormat(FileFormatOptions.AVRO); options.setOutputAsset(outputAsset.getName()); DataplexFileFormatConversion.run( @@ -306,9 +306,9 @@ public void testAssetWithEntityJsonToParquetSkipExistingFilesE2E() throws IOExce FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(asset2.getName()); - options.setOutputFileFormat(OutputFileFormat.PARQUET); + options.setOutputFileFormat(FileFormatOptions.PARQUET); options.setOutputAsset(outputAsset.getName()); - options.setWriteDisposition(ExistingOutputFilesBehaviour.SKIP); + options.setWriteDisposition(WriteDispositionOptions.SKIP); // simulate the file 1.json -> 1.parquet already copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/1.parquet"); @@ -363,9 +363,9 @@ public void testAssetWithEntityJsonToParquetFailOnExistingFilesE2E() throws IOEx FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(asset2.getName()); - options.setOutputFileFormat(OutputFileFormat.PARQUET); + options.setOutputFileFormat(FileFormatOptions.PARQUET); options.setOutputAsset(outputAsset.getName()); - options.setWriteDisposition(ExistingOutputFilesBehaviour.FAIL); + options.setWriteDisposition(WriteDispositionOptions.FAIL); // simulate the file 1.json -> 1.parquet already copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/1.parquet"); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java index 2f5373e364..3efeb678dc 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java @@ -15,8 +15,8 @@ */ package com.google.cloud.teleport.v2.transforms; -import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned.OutputFileFormat; import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned.PartitioningSchema; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.values.PartitionMetadata; import com.google.common.collect.ImmutableList; import java.time.ZoneOffset; @@ -81,7 +81,7 @@ public void testMonthlyPartitioning() { SERIALIZED_SCHEMA, "date", PartitioningSchema.MONTHLY, - OutputFileFormat.AVRO)); + FileFormatOptions.AVRO)); PAssert.that(result) .containsInAnyOrder( @@ -129,7 +129,7 @@ public void testDailyPartitioning() { SERIALIZED_SCHEMA, "date", PartitioningSchema.DAILY, - OutputFileFormat.AVRO)); + FileFormatOptions.AVRO)); PAssert.that(result) .containsInAnyOrder( @@ -182,7 +182,7 @@ public void testHourlyPartitioning() { SERIALIZED_SCHEMA, "date", PartitioningSchema.HOURLY, - OutputFileFormat.AVRO)); + FileFormatOptions.AVRO)); PAssert.that(result) .containsInAnyOrder( @@ -226,7 +226,7 @@ public void testParquet() { SERIALIZED_SCHEMA, "date", PartitioningSchema.MONTHLY, - OutputFileFormat.PARQUET)); + FileFormatOptions.PARQUET)); mainPipeline.run(); @@ -262,7 +262,7 @@ public void testNoPartitioning() { .apply( "GenericRecordsToGCS", new GenericRecordsToGcsPartitioned( - tmpRootPath, SERIALIZED_SCHEMA, null, null, OutputFileFormat.AVRO)); + tmpRootPath, SERIALIZED_SCHEMA, null, null, FileFormatOptions.AVRO)); mainPipeline.run(); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java index 3e11e8d222..efd01b4383 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexBigQueryToGcsFilterTest.java @@ -18,10 +18,10 @@ import static com.google.common.truth.Truth.assertThat; import com.google.cloud.teleport.v2.options.DataplexBigQueryToGcsOptions; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.FileFormat; -import com.google.cloud.teleport.v2.transforms.BigQueryTableToGcsTransform.WriteDisposition; import com.google.cloud.teleport.v2.utils.BigQueryMetadataLoader.Filter; -import com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.WriteDisposition.WriteDispositionOptions; import com.google.cloud.teleport.v2.values.BigQueryTable; import com.google.cloud.teleport.v2.values.BigQueryTablePartition; import java.io.IOException; @@ -237,8 +237,8 @@ public void test_whenTargetFileExistsWithWriteDispositionSKIP_filterExcludesTabl options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); - options.setFileFormat(FileFormat.AVRO); - options.setWriteDisposition(WriteDisposition.SKIP); + options.setFileFormat(FileFormatOptions.AVRO); + options.setWriteDisposition(WriteDispositionOptions.SKIP); Filter f = new DataplexBigQueryToGcsFilter( @@ -257,8 +257,8 @@ public void test_whenTargetFileExistsWithWriteDispositionOverwrite_filterAccepts options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); - options.setFileFormat(FileFormat.AVRO); - options.setWriteDisposition(WriteDisposition.OVERWRITE); + options.setFileFormat(FileFormatOptions.AVRO); + options.setWriteDisposition(WriteDispositionOptions.OVERWRITE); Filter f = new DataplexBigQueryToGcsFilter( @@ -277,8 +277,8 @@ public void test_whenTargetFileExistsWithWriteDispositionFail_filterAcceptsTable options.setTables(null); options.setExportDataModifiedBeforeDateTime(null); - options.setFileFormat(FileFormat.AVRO); - options.setWriteDisposition(WriteDisposition.FAIL); + options.setFileFormat(FileFormatOptions.AVRO); + options.setWriteDisposition(WriteDispositionOptions.FAIL); Filter f = new com.google.cloud.teleport.v2.utils.DataplexBigQueryToGcsFilter( options, diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/StorageUtilsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/StorageUtilsTest.java new file mode 100644 index 0000000000..a43ca8114b --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/StorageUtilsTest.java @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +import static com.google.common.truth.Truth.assertThat; + +import java.io.File; +import java.util.List; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link StorageUtils}. */ +@RunWith(JUnit4.class) +public final class StorageUtilsTest { + @Rule public final TemporaryFolder tmpDir = new TemporaryFolder(); + + @Test + public void testGetFilesInDirectory_withValidPath_returnsPathsOfFilesInDirectory() + throws Exception { + File outDir = tmpDir.newFolder("out"); + File outputDir1 = tmpDir.newFolder("out", "unpartitioned_table"); + File outputFile1 = + new File(outputDir1.getAbsolutePath() + "/" + "output-unpartitioned_table.parquet"); + outputFile1.createNewFile(); + File outputDir2 = tmpDir.newFolder("out", "partitioned_table", "p2_pid=partition"); + File outputFile2 = + new File(outputDir2.getAbsolutePath() + "/" + "output-partitioned_table-partition.parquet"); + outputFile2.createNewFile(); + + List files = StorageUtils.getFilesInDirectory(outDir.getAbsolutePath()); + assertThat(files.size()).isEqualTo(2); + } +} From 878964da147d4ac820751740e592b57ab56072dd Mon Sep 17 00:00:00 2001 From: olegsa Date: Thu, 17 Feb 2022 09:44:56 -0800 Subject: [PATCH 063/145] Refactor BigQueryTableToGcsTransform date type fix The CL updates how the dates in GenericRecords are transformed. This should not change the visible behavior of the transform. PiperOrigin-RevId: 429332100 --- .../transforms/BigQueryTableToGcsTransform.java | 15 +++++++++------ .../google/cloud/teleport/v2/utils/Schemas.java | 15 +++++++++++++++ 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java index 711aa2d004..022b38472c 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java @@ -15,7 +15,6 @@ */ package com.google.cloud.teleport.v2.transforms; -import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.cloud.teleport.v2.utils.BigQueryToGcsDirectoryNaming; import com.google.cloud.teleport.v2.utils.BigQueryToGcsFileNaming; import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; @@ -27,7 +26,9 @@ import java.time.LocalDate; import java.util.ArrayList; import java.util.List; +import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.coders.AvroCoder; @@ -234,16 +235,18 @@ private TypedRead getDefaultRead() { private GenericRecord genericRecordWithFixedDates(SchemaAndRecord schemaAndRecord) { GenericRecord input = schemaAndRecord.getRecord(); GenericRecord output = new GenericData.Record(table.getSchema()); - for (TableFieldSchema fieldSchema : schemaAndRecord.getTableSchema().getFields()) { - if ("DATE".equals(fieldSchema.getType())) { - Object value = input.get(fieldSchema.getName()); + List fields = table.getSchema().getFields(); + for (int i = 0; i < fields.size(); i++) { + if (Schemas.isSchemaOfTypeOrNullableType( + fields.get(i).schema(), Schema.Type.INT, LogicalTypes.date())) { + Object value = input.get(i); if (!(value instanceof CharSequence)) { throw new IllegalStateException( "The class of input value of type DATE is " + value.getClass()); } - output.put(fieldSchema.getName(), (int) LocalDate.parse((CharSequence) value).toEpochDay()); + output.put(i, (int) LocalDate.parse((CharSequence) value).toEpochDay()); } else { - output.put(fieldSchema.getName(), input.get(fieldSchema.getName())); + output.put(i, input.get(i)); } } return output; diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java index 585ef23525..bd00b9f4da 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java @@ -25,7 +25,9 @@ import java.sql.SQLException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import javax.sql.DataSource; +import org.apache.avro.LogicalType; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.SchemaBuilder; @@ -74,6 +76,19 @@ public static org.apache.beam.sdk.schemas.Schema jdbcSchemaToBeamSchema( } } + public static boolean isSchemaOfType(Schema schema, Schema.Type type, LogicalType logicalType) { + return Objects.equals(type, schema.getType()) + && Objects.equals(logicalType, schema.getLogicalType()); + } + + public static boolean isSchemaOfTypeOrNullableType( + Schema schema, Schema.Type type, LogicalType logicalType) { + return isSchemaOfType(schema, type, logicalType) + || Objects.equals(Schema.Type.UNION, schema.getType()) + && schema.getTypes().stream() + .anyMatch(t -> isSchemaOfTypeOrNullableType(t, type, logicalType)); + } + private static Schema dataplexFieldsToAvro( List dataplexFields, RecordBuilder avroRecordBuilder) From bf1f0c5d5d144f406109819eb9456b397e9e18c3 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 17 Feb 2022 16:34:18 -0500 Subject: [PATCH 064/145] Add some basic cleanup actions --- .github/actions/cleanup-java-env/action.yml | 35 +++++++++++++++++++++ .github/workflows/prepare-java-cache.yml | 3 ++ 2 files changed, 38 insertions(+) create mode 100644 .github/actions/cleanup-java-env/action.yml diff --git a/.github/actions/cleanup-java-env/action.yml b/.github/actions/cleanup-java-env/action.yml new file mode 100644 index 0000000000..a02a383088 --- /dev/null +++ b/.github/actions/cleanup-java-env/action.yml @@ -0,0 +1,35 @@ +# Copyright 2022 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 +# +# https://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. + +# Cleans up the Java environment. This is mostly intended to help keep the +# Java cache a bit smaller by removing unnecessary dependencies or stuff that +# we will always rebuild anyways. + +name: 'Cleanup Java Environment' +description: 'Do some cleanup of the Java environment' + +runs: + using: 'composite' + steps: + - name: Remove Teleport Artifacts + shell: bash + run: | + TELEPORT_DIR=~/.m2/repository/com/google/cloud/teleport + if [[ -d "$TELEPORT_DIR" ]]; then + echo "Cleaning up $TELEPORT_DIR" + rm -r "$TELEPORT_DIR" + echo "$TELEPORT_DIR successfully cleaned up" + else + echo "$TELEPORT_DIR not found" + fi diff --git a/.github/workflows/prepare-java-cache.yml b/.github/workflows/prepare-java-cache.yml index 62cc2f10b4..f111ed40dd 100644 --- a/.github/workflows/prepare-java-cache.yml +++ b/.github/workflows/prepare-java-cache.yml @@ -39,6 +39,7 @@ on: paths: - '.github/workflows/prepare-java-chace.yml' - '.github/actions/setup-java-env/*' + - '.github/actions/cleanup-java-env/*' permissions: read-all @@ -72,3 +73,5 @@ jobs: mvn -B clean install -f "$POMF" -am -amd -Dmaven.test.skip -Dcheckstyle.skip -Djib.skip -Dmdep.analyze.skip fi done + - name: Cleanup Java Environment + uses: ./.github/actions/cleanup-java-env From 9925548e134a7494451d167e978872936a6718ff Mon Sep 17 00:00:00 2001 From: dhercher Date: Fri, 25 Feb 2022 04:22:57 -0800 Subject: [PATCH 065/145] DatastreamToSQL: Adding support for custom connection strings to allow SSL PiperOrigin-RevId: 430917033 --- .../cloud/teleport/v2/templates/DataStreamToSQL.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java index a0c52930eb..74e85e98bd 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java @@ -156,6 +156,12 @@ public interface Options extends PipelineOptions, StreamingOptions { String getSchemaMap(); void setSchemaMap(String value); + + @Description("[Optional] Custom connection string") + @Default.String("") + String getCustomConnectionString(); + + void setCustomConnectionString(String value); } /** @@ -164,7 +170,7 @@ public interface Options extends PipelineOptions, StreamingOptions { * @param args The command-line arguments to the pipeline. */ public static void main(String[] args) { - LOG.info("Starting Avro Python to BigQuery"); + LOG.info("Starting Datastream to SQL"); Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); @@ -201,6 +207,9 @@ public static CdcJdbcIO.DataSourceConfiguration getDataSourceConfiguration(Optio throw new IllegalArgumentException( String.format("Database Type %s is not supported.", options.getDatabaseType())); } + if (!options.getCustomConnectionString().isEmpty()) { + jdbcDriverConnectionString = options.getCustomConnectionString(); + } CdcJdbcIO.DataSourceConfiguration dataSourceConfiguration = CdcJdbcIO.DataSourceConfiguration.create(jdbcDriverName, jdbcDriverConnectionString) From 9916209f4548bae3c56a1476895d6735ebb6c17a Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Wed, 2 Mar 2022 09:54:18 -0800 Subject: [PATCH 066/145] Add metrics to improve observability of PubSubToSplunk template. PiperOrigin-RevId: 431963950 --- .../teleport/splunk/SplunkEventWriter.java | 116 +++++++++++++++--- .../cloud/teleport/splunk/SplunkIO.java | 29 ++++- .../teleport/templates/PubSubToSplunk.java | 1 + .../templates/common/SplunkConverters.java | 7 ++ 4 files changed, 133 insertions(+), 20 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java b/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java index 1ab4726021..9b61f2400e 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java +++ b/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java @@ -34,11 +34,13 @@ import java.security.KeyStoreException; import java.security.NoSuchAlgorithmException; import java.security.cert.CertificateException; +import java.time.Instant; import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.annotation.Nullable; import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.state.BagState; @@ -62,6 +64,7 @@ public abstract class SplunkEventWriter extends DoFn, S private static final Integer DEFAULT_BATCH_COUNT = 1; private static final Boolean DEFAULT_DISABLE_CERTIFICATE_VALIDATION = false; + private static final Boolean DEFAULT_ENABLE_BATCH_LOGS = true; private static final Logger LOG = LoggerFactory.getLogger(SplunkEventWriter.class); private static final long DEFAULT_FLUSH_DELAY = 2; private static final Counter INPUT_COUNTER = @@ -70,6 +73,18 @@ public abstract class SplunkEventWriter extends DoFn, S Metrics.counter(SplunkEventWriter.class, "outbound-successful-events"); private static final Counter FAILED_WRITES = Metrics.counter(SplunkEventWriter.class, "outbound-failed-events"); + private static final Counter INVALID_REQUESTS = + Metrics.counter(SplunkEventWriter.class, "http-invalid-requests"); + private static final Counter SERVER_ERROR_REQUESTS = + Metrics.counter(SplunkEventWriter.class, "http-server-error-requests"); + private static final Counter VALID_REQUESTS = + Metrics.counter(SplunkEventWriter.class, "http-valid-requests"); + private static final Distribution SUCCESSFUL_WRITE_LATENCY_MS = + Metrics.distribution(SplunkEventWriter.class, "successful_write_to_splunk_latency_ms"); + private static final Distribution UNSUCCESSFUL_WRITE_LATENCY_MS = + Metrics.distribution(SplunkEventWriter.class, "unsuccessful_write_to_splunk_latency_ms"); + private static final Distribution SUCCESSFUL_WRITE_BATCH_SIZE = + Metrics.distribution(SplunkEventWriter.class, "write_to_splunk_batch"); private static final String BUFFER_STATE_NAME = "buffer"; private static final String COUNT_STATE_NAME = "count"; private static final String TIME_ID_NAME = "expiry"; @@ -91,6 +106,7 @@ public abstract class SplunkEventWriter extends DoFn, S private Integer batchCount; private Boolean disableValidation; + private Boolean enableBatchLogs; private HttpEventPublisher publisher; private static final Gson GSON = @@ -112,6 +128,9 @@ public static Builder newBuilder() { @Nullable abstract ValueProvider rootCaCertificatePath(); + @Nullable + abstract ValueProvider enableBatchLogs(); + @Nullable abstract ValueProvider inputBatchCount(); @@ -133,6 +152,16 @@ public void setup() { LOG.info("Batch count set to: {}", batchCount); } + if (enableBatchLogs == null) { + + if (enableBatchLogs() != null) { + enableBatchLogs = enableBatchLogs().get(); + } + + enableBatchLogs = MoreObjects.firstNonNull(enableBatchLogs, DEFAULT_ENABLE_BATCH_LOGS); + LOG.info("Enable Batch logs set to: {}", enableBatchLogs); + } + // Either user supplied or default disableValidation. if (disableValidation == null) { @@ -188,8 +217,9 @@ public void processElement( timer.offset(Duration.standardSeconds(DEFAULT_FLUSH_DELAY)).setRelative(); if (count >= batchCount) { - - LOG.info("Flushing batch of {} events", count); + if (enableBatchLogs) { + LOG.info("Flushing batch of {} events", count); + } flush(receiver, bufferState, countState); } } @@ -202,7 +232,9 @@ public void onExpiry( throws IOException { if (MoreObjects.firstNonNull(countState.read(), 0L) > 0) { - LOG.info("Flushing window with {} events", countState.read()); + if (enableBatchLogs) { + LOG.info("Flushing window with {} events", countState.read()); + } flush(receiver, bufferState, countState); } } @@ -221,7 +253,7 @@ public void tearDown() { } /** - * Utility method to flush a batch of requests via {@link HttpEventPublisher}. + * Utility method to flush a batch of events via {@link HttpEventPublisher}. * * @param receiver Receiver to write {@link SplunkWriteError}s to */ @@ -235,34 +267,58 @@ private void flush( HttpResponse response = null; List events = Lists.newArrayList(bufferState.read()); + long startTime = System.nanoTime(); try { // Important to close this response to avoid connection leak. response = publisher.execute(events); - if (!response.isSuccessStatusCode()) { + UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + FAILED_WRITES.inc(countState.read()); + int statusCode = response.getStatusCode(); + if (statusCode >= 400 && statusCode < 500) { + INVALID_REQUESTS.inc(); + } else if (statusCode >= 500 && statusCode < 600) { + SERVER_ERROR_REQUESTS.inc(); + } + + logWriteFailures( + countState, + response.getStatusCode(), + response.parseAsString(), + response.getStatusMessage()); flushWriteFailures( events, response.getStatusMessage(), response.getStatusCode(), receiver); - logWriteFailures(countState); } else { - LOG.info("Successfully wrote {} events", countState.read()); + SUCCESSFUL_WRITE_LATENCY_MS.update(Instant.now().toEpochMilli() - startTime); SUCCESS_WRITES.inc(countState.read()); + VALID_REQUESTS.inc(); + SUCCESSFUL_WRITE_BATCH_SIZE.update(countState.read()); + + if (enableBatchLogs) { + LOG.info("Successfully wrote {} events", countState.read()); + } } } catch (HttpResponseException e) { - LOG.error( - "Error writing to Splunk. StatusCode: {}, content: {}, StatusMessage: {}", - e.getStatusCode(), - e.getContent(), - e.getStatusMessage()); - logWriteFailures(countState); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + FAILED_WRITES.inc(countState.read()); + int statusCode = e.getStatusCode(); + if (statusCode >= 400 && statusCode < 500) { + INVALID_REQUESTS.inc(); + } else if (statusCode >= 500 && statusCode < 600) { + SERVER_ERROR_REQUESTS.inc(); + } + logWriteFailures(countState, e.getStatusCode(), e.getContent(), e.getStatusMessage()); flushWriteFailures(events, e.getStatusMessage(), e.getStatusCode(), receiver); } catch (IOException ioe) { - LOG.error("Error writing to Splunk: {}", ioe.getMessage()); - logWriteFailures(countState); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + FAILED_WRITES.inc(countState.read()); + INVALID_REQUESTS.inc(); + logWriteFailures(countState, 0, ioe.getMessage(), null); flushWriteFailures(events, ioe.getMessage(), null, receiver); } finally { @@ -293,10 +349,20 @@ private void flush( } } - /** Utility method to log write failures and handle metrics. */ - private void logWriteFailures(@StateId(COUNT_STATE_NAME) ValueState countState) { - LOG.error("Failed to write {} events", countState.read()); - FAILED_WRITES.inc(countState.read()); + /** Utility method to log write failures. */ + private void logWriteFailures( + @StateId(COUNT_STATE_NAME) ValueState countState, + int statusCode, + String content, + String statusMessage) { + if (enableBatchLogs) { + LOG.error("Failed to write {} events", countState.read()); + } + LOG.error( + "Error writing to Splunk. StatusCode: {}, content: {}, StatusMessage: {}", + statusCode, + content, + statusMessage); } /** @@ -364,6 +430,8 @@ abstract Builder setDisableCertificateValidation( abstract Builder setRootCaCertificatePath(ValueProvider rootCaCertificatePath); + abstract Builder setEnableBatchLogs(ValueProvider enableBatchLogs); + abstract Builder setInputBatchCount(ValueProvider inputBatchCount); abstract SplunkEventWriter autoBuild(); @@ -447,6 +515,16 @@ public Builder withRootCaCertificatePath(ValueProvider rootCaCertificate return setRootCaCertificatePath(rootCaCertificatePath); } + /** + * Method to enable batch logs. + * + * @param enableBatchLogs for enabling batch logs. + * @return {@link Builder} + */ + public Builder withEnableBatchLogs(ValueProvider enableBatchLogs) { + return setEnableBatchLogs(enableBatchLogs); + } + /** Build a new {@link SplunkEventWriter} objects based on the configuration. */ public SplunkEventWriter build() { checkNotNull(url(), "url needs to be provided."); diff --git a/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java b/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java index 610aae8c48..6f34649946 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java +++ b/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java @@ -78,6 +78,9 @@ public abstract static class Write @Nullable abstract ValueProvider rootCaCertificatePath(); + @Nullable + abstract ValueProvider enableBatchLogs(); + @Override public PCollection expand(PCollection input) { @@ -88,7 +91,8 @@ public PCollection expand(PCollection input) { .withInputBatchCount(batchCount()) .withDisableCertificateValidation(disableCertificateValidation()) .withToken((token())) - .withRootCaCertificatePath(rootCaCertificatePath()); + .withRootCaCertificatePath(rootCaCertificatePath()) + .withEnableBatchLogs(enableBatchLogs()); SplunkEventWriter writer = builder.build(); LOG.info("SplunkEventWriter configured"); @@ -121,6 +125,8 @@ abstract Builder setDisableCertificateValidation( abstract Builder setRootCaCertificatePath(ValueProvider rootCaCertificatePath); + abstract Builder setEnableBatchLogs(ValueProvider enableBatchLogs); + abstract Write autoBuild(); /** @@ -268,6 +274,27 @@ public Builder withRootCaCertificatePath(String rootCaCertificatePath) { ValueProvider.StaticValueProvider.of(rootCaCertificatePath)); } + /** + * Method to enable batch logs. + * + * @param enableBatchLogs for enabling batch logs. + * @return {@link Builder} + */ + public Builder withEnableBatchLogs(ValueProvider enableBatchLogs) { + return setEnableBatchLogs(enableBatchLogs); + } + + /** + * Same as {@link Builder#withEnableBatchLogs(ValueProvider)} but without a {@link + * ValueProvider}. + * + * @param enableBatchLogs for enabling batch logs. + * @return {@link Builder} + */ + public Builder withEnableBatchLogs(Boolean enableBatchLogs) { + return setEnableBatchLogs(ValueProvider.StaticValueProvider.of((enableBatchLogs))); + } + public Write build() { checkNotNull(url(), "HEC url is required."); checkNotNull(token(), "Authorization token is required."); diff --git a/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java b/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java index c099f11350..43b6a221c9 100644 --- a/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java +++ b/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java @@ -251,6 +251,7 @@ public static PipelineResult run(PubSubToSplunkOptions options) { .withParallelism(options.getParallelism()) .withDisableCertificateValidation(options.getDisableCertificateValidation()) .withRootCaCertificatePath(options.getRootCaCertificatePath()) + .withEnableBatchLogs(options.getEnableBatchLogs()) .build()); // 5a) Wrap write failures into a FailsafeElement. diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java index 94e11f7029..c921d09fbb 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java @@ -137,6 +137,13 @@ public interface SplunkOptions extends PipelineOptions { ValueProvider getRootCaCertificatePath(); void setRootCaCertificatePath(ValueProvider rootCaPath); + + @Description( + "Parameter which specifies if logs should be enabled for batches written to Splunk." + + " Default: true.") + ValueProvider getEnableBatchLogs(); + + void setEnableBatchLogs(ValueProvider enableBatchLogs); } private static class FailsafeStringToSplunkEvent From a1ecc42b726398748530983cc881ddb0491d4848 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Mon, 7 Mar 2022 13:36:57 -0800 Subject: [PATCH 067/145] Add an aggregator POM to help build and test automations. PiperOrigin-RevId: 433027177 --- pom.xml | 2 +- unified-templates.xml | 59 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 60 insertions(+), 1 deletion(-) create mode 100644 unified-templates.xml diff --git a/pom.xml b/pom.xml index 557dfecaa7..1f4cd1cc33 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ com.google.cloud.teleport google-cloud-teleport-java - 0.1-SNAPSHOT + 0.1.0-SNAPSHOT Google Cloud Teleport Cloud Teleport is a collection of Apache Beam pipelines for common tasks, like data movement diff --git a/unified-templates.xml b/unified-templates.xml new file mode 100644 index 0000000000..a2c36f848b --- /dev/null +++ b/unified-templates.xml @@ -0,0 +1,59 @@ + + + + 4.0.0 + + com.google.cloud.teleport + unified-templates + 1.0-SNAPSHOT + Google Cloud Dataflow Templates + pom + + Unifies all the template libraries under one POM for easier automation and to allow + for more reusable and modular dependencies. + + + + 3.0.1 + + + + + + + maven-source-plugin + ${maven-source-plugin.version} + + + attach-sources + + jar + + + + + + + + + + . + v2 + + \ No newline at end of file From 5822c3634c4660573100d55d4da2b872e6c67b27 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Sat, 22 Jan 2022 14:58:01 -0500 Subject: [PATCH 068/145] Add integration testing library to make writing tests easier --- example-input.txt | 2 + v2/pom.xml | 7 + v2/streaming-data-generator/pom.xml | 6 + .../templates/StreamingDataGeneratorIT.java | 46 +++ v2/testing/pom.xml | 39 +++ .../teleport/v2/testing/TestProperties.java | 73 +++++ .../v2/testing/artifacts/ArtifactClient.java | 14 + .../artifacts/ArtifactGcsSdkClient.java | 90 ++++++ .../v2/testing/artifacts/ArtifactUtils.java | 21 ++ .../v2/testing/artifacts/package-info.java | 1 + .../testing/dataflow/DataflowOperation.java | 148 +++++++++ .../v2/testing/dataflow/DataflowUtils.java | 15 + .../testing/dataflow/FlexTemplateClient.java | 99 ++++++ .../dataflow/FlexTemplateSdkClient.java | 96 ++++++ .../v2/testing/dataflow/package-info.java | 1 + .../src/main/resources/test-artifact.txt | 1 + .../v2/testing/TestPropertiesTest.java | 78 +++++ .../artifacts/ArtifactGcsSdkClientTest.java | 297 ++++++++++++++++++ .../testing/artifacts/ArtifactUtilsTest.java | 37 +++ .../dataflow/DataflowOperationTest.java | 177 +++++++++++ .../testing/dataflow/DataflowUtilsTest.java | 18 ++ .../dataflow/FlexTemplateSdkClientTest.java | 157 +++++++++ 22 files changed, 1423 insertions(+) create mode 100644 example-input.txt create mode 100644 v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java create mode 100644 v2/testing/pom.xml create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java create mode 100644 v2/testing/src/main/resources/test-artifact.txt create mode 100644 v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java create mode 100644 v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java create mode 100644 v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java create mode 100644 v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java create mode 100644 v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java create mode 100644 v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java diff --git a/example-input.txt b/example-input.txt new file mode 100644 index 0000000000..cf77958798 --- /dev/null +++ b/example-input.txt @@ -0,0 +1,2 @@ +hello1 +hello2 diff --git a/v2/pom.xml b/v2/pom.xml index 54b3cc7bd1..f3a822974f 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -49,6 +49,7 @@ 3.0.1 3.0.0 1.5.0.Final + 1.5 1.7.25 ${basedir}/target 20200518 @@ -172,6 +173,11 @@ auto-value-annotations ${autovalue.annotations.version} + + com.google.re2j + re2j + ${re2j.version} + org.slf4j slf4j-simple @@ -670,6 +676,7 @@ streaming-data-generator kafka-to-pubsub kafka-common + testing diff --git a/v2/streaming-data-generator/pom.xml b/v2/streaming-data-generator/pom.xml index 7444b2bd1e..c73de3fcd5 100644 --- a/v2/streaming-data-generator/pom.xml +++ b/v2/streaming-data-generator/pom.xml @@ -62,6 +62,12 @@ snakeyaml ${snakeyaml.version} + + com.google.cloud.teleport.v2 + testing + 1.0-SNAPSHOT + test + diff --git a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java new file mode 100644 index 0000000000..fe2a21d520 --- /dev/null +++ b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java @@ -0,0 +1,46 @@ +package com.google.cloud.teleport.v2.templates; + +import com.google.cloud.teleport.v2.templates.StreamingDataGenerator.SinkType; +import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobInfo; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.Options; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateSdkClient; +import java.io.IOException; +import java.time.Duration; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public final class StreamingDataGeneratorIT { + + private static final Duration MAX_WAIT_TIME = Duration.ofMinutes(15); + + private static final String NUM_SHARDS_KEY = "numShards"; + private static final String OUTPUT_DIRECTORY_KEY = "outputDirectory"; + private static final String QPS_KEY = "qps"; + private static final String SCHEMA_LOCATION_KEY = "schemaLocation"; + private static final String SINK_TYPE_KEY = "sinkType"; + private static final String WINDOW_DURATION_KEY = "windowDuration"; + + private static final String DEFAULT_QPS = "15"; + private static final String DEFAULT_WINDOW_DURATION = "60s"; + + @Test + public void testFakeMessagesToGcs() throws IOException { + String name = "teleport_flex_streaming_data_generator_gcs"; + Options options = new Options(name, "TODO") + .setIsStreaming(true) + .addParameter(SCHEMA_LOCATION_KEY, "TODO") + .addParameter(QPS_KEY, DEFAULT_QPS) + .addParameter(SINK_TYPE_KEY, SinkType.GCS.name()) + .addParameter(WINDOW_DURATION_KEY, DEFAULT_WINDOW_DURATION) + .addParameter(OUTPUT_DIRECTORY_KEY, "TODO") + .addParameter(NUM_SHARDS_KEY, "1"); + FlexTemplateClient dataflow = FlexTemplateSdkClient.builder().build(); + + JobInfo info = dataflow.launchNewJob("TODO", "TODO", options); + DataflowOperation.waitForConditionAndFinish(dataflow, null, () -> false); + } +} diff --git a/v2/testing/pom.xml b/v2/testing/pom.xml new file mode 100644 index 0000000000..9ea9d2ef61 --- /dev/null +++ b/v2/testing/pom.xml @@ -0,0 +1,39 @@ + + + testing + 4.0.0 + + + dynamic-templates + com.google.cloud.teleport.v2 + 1.0-SNAPSHOT + + + + 2.5.0 + 1.0.1 + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${dataflow-client.version} + + + com.google.cloud + google-cloud-storage + + + + + com.google.truth + truth + ${truth.version} + test + + + + \ No newline at end of file diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java new file mode 100644 index 0000000000..01baaeb503 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java @@ -0,0 +1,73 @@ +package com.google.cloud.teleport.v2.testing; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; + +import java.util.HashMap; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility for accessing system properties set for the test. + * + *

The values should be passed to the test like `-Dkey=value`. For instance, + * `-Dproject=my-project`. + */ +public final class TestProperties { + private static final Logger LOG = LoggerFactory.getLogger(TestProperties.class); + + public static final String ACCESS_TOKEN_KEY = "accessToken"; + public static final String ARTIFACT_BUCKET_KEY = "artifactBucket"; + public static final String PROJECT_KEY = "project"; + public static final String SPEC_PATH_KEY = "specPath"; + + private static String accessToken; + private static String artifactBucket; + private static String project; + private static String specPath; + + private final Map initialized; + + public TestProperties() { + initialized = new HashMap<>(); + initialized.put(ACCESS_TOKEN_KEY, false); + initialized.put(ARTIFACT_BUCKET_KEY, false); + initialized.put(PROJECT_KEY, false); + initialized.put(SPEC_PATH_KEY, false); + } + + public String accessToken() { + if (!initialized.get(ACCESS_TOKEN_KEY)) { + accessToken = System.getProperty(ACCESS_TOKEN_KEY, null); + initialized.replace(ACCESS_TOKEN_KEY, true); + } + return accessToken; + } + + public String artifactBucket() { + if (!initialized.get(ARTIFACT_BUCKET_KEY)) { + artifactBucket = System.getProperty(ARTIFACT_BUCKET_KEY, null); + checkState(artifactBucket != null, "%s is required", ARTIFACT_BUCKET_KEY); + initialized.replace(ARTIFACT_BUCKET_KEY, true); + } + return artifactBucket; + } + + public String project() { + if (!initialized.get(PROJECT_KEY)) { + project = System.getProperty(PROJECT_KEY, null); + checkState(project != null, "%s is required", PROJECT_KEY); + initialized.replace(PROJECT_KEY, true); + } + return project; + } + + public String specPath() { + if (!initialized.get(SPEC_PATH_KEY)) { + specPath = System.getProperty(SPEC_PATH_KEY, null); + checkState(specPath != null, "%s is required", SPEC_PATH_KEY); + initialized.replace(SPEC_PATH_KEY, true); + } + return specPath; + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java new file mode 100644 index 0000000000..bb4932948d --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java @@ -0,0 +1,14 @@ +package com.google.cloud.teleport.v2.testing.artifacts; + +import com.google.cloud.storage.Blob; +import com.google.re2j.Pattern; +import java.io.IOException; +import java.util.List; + +public interface ArtifactClient { + Blob uploadArtifact(String bucket, String gcsPath, String localPath) throws IOException; + + List listArtifacts(String bucket, String testDirPath, Pattern regex); + + void deleteTestDir(String bucket, String testDirPath); +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java new file mode 100644 index 0000000000..661eaa7bd3 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java @@ -0,0 +1,90 @@ +package com.google.cloud.teleport.v2.testing.artifacts; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList.toImmutableList; + +import com.google.api.gax.paging.Page; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.Storage.BlobListOption; +import com.google.re2j.Pattern; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; +import java.util.stream.StreamSupport; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Client for working with test artifacts which uses the GCS SDK. */ +public final class ArtifactGcsSdkClient implements ArtifactClient { + private static final Logger LOG = LoggerFactory.getLogger(ArtifactGcsSdkClient.class); + + private final Storage client; + + public ArtifactGcsSdkClient(Storage client) { + this.client = client; + } + + public Blob uploadArtifact(String bucket, String gcsPath, String localPath) + throws IOException { + BlobId id = BlobId.of(bucket, gcsPath); + BlobInfo info = BlobInfo.newBuilder(id).build(); + + byte[] contents = Files.readAllBytes(Paths.get(localPath)); + + return client.create(info, contents); + } + + public List listArtifacts(String bucket, String testDirPath, Pattern regex) { + List result = new ArrayList<>(); + consumeTestDir(bucket, testDirPath, blobs -> { + for (Blob blob : blobs) { + if (regex.matches(blob.getName())) { + result.add(blob); + } + } + }); + return result; + } + + public void deleteTestDir(String bucket, String testDirPath) { + consumeTestDir(bucket, testDirPath, blobs -> { + // Go through the Iterable overload, since the other ones make it very difficult to + // do thorough testing with Mockito + ImmutableList blobIds = StreamSupport.stream(blobs.spliterator(), false) + .map(Blob::getBlobId) + .collect(toImmutableList()); + if (blobIds.isEmpty()) { + return; + } + List deleted = client.delete(blobIds); + for (int i = 0; i < blobIds.size(); ++i) { + if (!deleted.get(i)) { + LOG.warn("Blob {} not deleted", blobIds.get(i).getName()); + } + } + }); + } + + private void consumeTestDir(String bucket, String testDirPath, Consumer> consumeBlobs) { + Page blobs = getFirstTestDirPage(bucket, testDirPath); + while (true) { + consumeBlobs.accept(blobs.getValues()); + + if (blobs.hasNextPage()) { + blobs = blobs.getNextPage(); + } else { + break; + } + } + } + + private Page getFirstTestDirPage(String bucket, String testDirPath) { + return client.list(bucket, BlobListOption.prefix(testDirPath)); + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java new file mode 100644 index 0000000000..766d97f44f --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java @@ -0,0 +1,21 @@ +package com.google.cloud.teleport.v2.testing.artifacts; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.UUID; + +/** Utilities for working with test artifacts. */ +public final class ArtifactUtils { + private ArtifactUtils() {} + + public static String createTestDirName() { + return String.format("%s-%s", + DateTimeFormatter.ofPattern("yyyyMMdd").withZone(ZoneId.of("UTC")).format(Instant.now()), + UUID.randomUUID()); + } + + public static String createTestDirPath(String bucket, String suiteDir, String testDir) { + return String.format("gs://%s/%s/%s/%s", bucket, suiteDir, testDir, createTestDirName()); + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java new file mode 100644 index 0000000000..671b9c8243 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java @@ -0,0 +1 @@ +package com.google.cloud.teleport.v2.testing.artifacts; \ No newline at end of file diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java new file mode 100644 index 0000000000..03ec4ed159 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java @@ -0,0 +1,148 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import com.google.auto.value.AutoValue; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; +import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.util.function.Supplier; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class DataflowOperation { + private static final Logger LOG = LoggerFactory.getLogger(DataflowOperation.class); + + public enum Result { + CONDITION_MET, + JOB_FINISHED, + TIMEOUT + } + + public static final ImmutableSet DONE_STATES = ImmutableSet.of( + JobState.CANCELLED, + JobState.DONE, + JobState.DRAINED, + JobState.FAILED, + JobState.STOPPED); + + public static final ImmutableSet FINISHING_STATES = ImmutableSet.of( + JobState.DRAINING, + JobState.CANCELLING); + + private DataflowOperation() {} + + public static Result waitUntilDone(FlexTemplateClient client, Config config) { + return finishOrTimeout(config, () -> false, () -> jobIsDone(client, config.project(), config.jobId())); + } + + public static Result waitForCondition(FlexTemplateClient client, Config config, Supplier conditionCheck) { + return finishOrTimeout(config, conditionCheck, () -> jobIsDoneOrFinishing(client, + config.project(), config.jobId())); + } + + public static Result waitForConditionAndFinish(FlexTemplateClient client, Config config, Supplier conditionCheck) + throws IOException { + Instant start = Instant.now(); + Result conditionStatus = waitForCondition(client, config, conditionCheck); + if (conditionStatus != Result.JOB_FINISHED) { + client.cancelJob(config.project(), config.jobId()); + waitUntilDone(client, config); + } + return conditionStatus; + } + + private static Result finishOrTimeout(Config config, Supplier conditionCheck, Supplier stopChecking) { + Instant start = Instant.now(); + + LOG.info("Making initial finish check."); + if (conditionCheck.get()) { + return Result.CONDITION_MET; + } + + LOG.info("Job was not already finished. Starting to wait between requests."); + while (timeIsLeft(start, config.timeoutAfter())) { + try { + Thread.sleep(config.checkAfter().toMillis()); + } catch (InterruptedException e) { + LOG.warn("Wait interrupted. Checking now."); + } + + LOG.info("Checking if condition is met."); + if (conditionCheck.get()) { + return Result.CONDITION_MET; + } + LOG.info("Condition not met. Checking if job is finished."); + if (stopChecking.get()) { + return Result.JOB_FINISHED; + } + LOG.info("Job not finished. Will check again in {} seconds", config.checkAfter().getSeconds()); + } + + LOG.warn("Neither the condition or job completion were fulfilled on time."); + return Result.TIMEOUT; + } + + private static boolean jobIsDone(FlexTemplateClient client, String project, String jobId) { + try { + JobState state = client.getJobStatus(project, jobId); + LOG.info("Job is in state {}", state); + return DONE_STATES.contains(state); + } catch (IOException e) { + LOG.error("Failed to get current job state. Assuming not done.", e); + return false; + } + } + + private static boolean jobIsDoneOrFinishing(FlexTemplateClient client, String project, String jobId) { + try { + JobState state = client.getJobStatus(project, jobId); + LOG.info("Job is in state {}", state); + return DONE_STATES.contains(state) || FINISHING_STATES.contains(state); + } catch (IOException e) { + LOG.error("Failed to get current job state. Assuming not done.", e); + return false; + } + } + + private static boolean timeIsLeft(Instant start, Duration maxWaitTime) { + return Duration.between(start, Instant.now()).minus(maxWaitTime).isNegative(); + } + + @AutoValue + public static abstract class Config { + public abstract String project(); + public abstract String jobId(); + public abstract Duration checkAfter(); + public abstract Duration timeoutAfter(); + // TODO(zhoufek): Also let users set the maximum number of exceptions. + + public static Builder builder() { + return new AutoValue_DataflowOperation_Config.Builder() + .setCheckAfter(Duration.ofSeconds(30)) + .setTimeoutAfter(Duration.ofMinutes(15)); + } + + @AutoValue.Builder + public static abstract class Builder { + public abstract Builder setProject(String value); + public abstract Builder setJobId(String value); + public abstract Builder setCheckAfter(Duration value); + public abstract Builder setTimeoutAfter(Duration value); + + abstract Config autoBuild(); + + public Config build() { + Config config = autoBuild(); + if (Strings.isNullOrEmpty(config.project())) { + throw new IllegalStateException("Project cannot be null or empty"); + } + if (Strings.isNullOrEmpty(config.jobId())) { + throw new IllegalStateException("Job ID cannot be null or empty"); + } + return config; + } + } + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java new file mode 100644 index 0000000000..dfc8773dcd --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java @@ -0,0 +1,15 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; + +public final class DataflowUtils { + private DataflowUtils() {} + + public static String createJobName(String prefix) { + return String.format("%s-%s", + prefix, + DateTimeFormatter.ofPattern("yyyyMMddHHmmss").withZone(ZoneId.of("UTC")).format(Instant.now())); + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java new file mode 100644 index 0000000000..9aaa62f0e2 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java @@ -0,0 +1,99 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public interface FlexTemplateClient { + enum JobState { + UNKNOWN("JOB_STATE_UNKNOWN"), + STOPPED("JOB_STATE_STOPPED"), + RUNNING("JOB_STATE_RUNNING"), + DONE("JOB_STATE_DONE"), + FAILED("JOB_STATE_FAILED"), + CANCELLED("JOB_STATE_CANCELLED"), + UPDATED("JOB_STATE_UPDATED"), + DRAINING("JOB_STATE_DRAINING"), + DRAINED("JOB_STATE_DRAINED"), + PENDING("JOB_STATE_PENDING"), + CANCELLING("JOB_STATE_CANCELLING"), + QUEUED("JOB_STATE_QUEUED"), + RESOURCE_CLEANING_UP("JOB_STATE_RESOURCE_CLEANING_UP"); + + private static final String DATAFLOW_PREFIX = "JOB_STATE_"; + + private final String text; + + JobState(String text) { + this.text = text; + } + + public static JobState parse(String fromDataflow) { + return valueOf(fromDataflow.replace(DATAFLOW_PREFIX, "")); + } + + @Override + public String toString() { + return text; + } + } + + class Options { + private final String jobName; + private final Map parameters; + private final String specPath; + + public Options(String jobName, String specPath) { + this.jobName = jobName; + this.specPath = specPath; + this.parameters = new HashMap<>(); + } + + public Options addParameter(String key, String value) { + parameters.put(key, value); + return this; + } + + public Options setIsStreaming(boolean value) { + return addParameter("isStreaming", Boolean.toString(value)); + } + + public String jobName() { + return jobName; + } + + public ImmutableMap parameters() { + return ImmutableMap.copyOf(parameters); + } + + public String specPath() { + return specPath; + } + } + + @AutoValue + abstract class JobInfo { + public abstract String jobId(); + + public abstract JobState state(); + + public static Builder builder() { + return new AutoValue_FlexTemplateClient_JobInfo.Builder(); + } + + @AutoValue.Builder + public static abstract class Builder { + public abstract Builder setJobId(String value); + public abstract Builder setState(JobState value); + public abstract JobInfo build(); + } + } + + JobInfo launchNewJob(String project, String region, Options options) throws IOException; + + JobState getJobStatus(String project, String jobId) throws IOException; + + void cancelJob(String project, String jobId) throws IOException; +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java new file mode 100644 index 0000000000..a92aa2bf9d --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java @@ -0,0 +1,96 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import com.google.api.client.googleapis.util.Utils; +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.LaunchFlexTemplateParameter; +import com.google.api.services.dataflow.model.LaunchFlexTemplateRequest; +import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse; +import com.google.auth.Credentials; +import com.google.auth.http.HttpCredentialsAdapter; +import java.io.IOException; +import java.time.Duration; + +/** Client for interacting with Dataflow Flex Templates using the Dataflow SDK. */ +public final class FlexTemplateSdkClient implements FlexTemplateClient { + + private final Dataflow client; + + // TODO(zhoufek): Let users set this in options. + private static final Duration REQUEST_TIMEOUT = Duration.ofSeconds(30); + + private FlexTemplateSdkClient(Builder builder) { + HttpRequestInitializer initializer = builder.getCredentials() != null + ? new HttpCredentialsAdapter(builder.getCredentials()) + : request -> { + request.setConnectTimeout((int) REQUEST_TIMEOUT.toMillis()); + request.setReadTimeout((int) REQUEST_TIMEOUT.toMillis()); + request.setWriteTimeout((int) REQUEST_TIMEOUT.toMillis()); + }; + + this.client = new Dataflow( + Utils.getDefaultTransport(), + Utils.getDefaultJsonFactory(), + initializer); + } + + private FlexTemplateSdkClient(Dataflow dataflow) { + this.client = dataflow; + } + + public static FlexTemplateSdkClient withDataflowClient(Dataflow dataflow) { + return new FlexTemplateSdkClient(dataflow); + } + + public static Builder builder() { + return new Builder(); + } + + public JobInfo launchNewJob(String project, String region, Options options) throws IOException { + LaunchFlexTemplateRequest request = new LaunchFlexTemplateRequest(); + LaunchFlexTemplateParameter parameter = new LaunchFlexTemplateParameter() + .setJobName(options.jobName()) + .setParameters(options.parameters()) + .setContainerSpecGcsPath(options.specPath()); + LaunchFlexTemplateResponse response = client.projects() + .locations() + .flexTemplates() + .launch(project, region, request.setLaunchParameter(parameter)) + .execute(); + Job job = response.getJob(); + return JobInfo.builder() + .setJobId(job.getId()) + .setState(JobState.parse(job.getCurrentState())) + .build(); + } + + public JobState getJobStatus(String project, String jobId) throws IOException { + Job job = client.projects().jobs().get(project, jobId).execute(); + return JobState.parse(job.getCurrentState()); + } + + public void cancelJob(String project, String jobId) throws IOException { + Job job = new Job().setCurrentState(JobState.CANCELLED.toString()); + client.projects().jobs().update(project, jobId, job).execute(); + } + + public static final class Builder { + private Credentials credentials; + + private Builder() {} + + public Credentials getCredentials() { + return credentials; + } + + public Builder setCredentials(Credentials value) { + credentials = value; + return this; + } + + public FlexTemplateSdkClient build() { + return new FlexTemplateSdkClient(this); + } + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java new file mode 100644 index 0000000000..bc90be24d7 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java @@ -0,0 +1 @@ +package com.google.cloud.teleport.v2.testing.dataflow; \ No newline at end of file diff --git a/v2/testing/src/main/resources/test-artifact.txt b/v2/testing/src/main/resources/test-artifact.txt new file mode 100644 index 0000000000..22c4e1d122 --- /dev/null +++ b/v2/testing/src/main/resources/test-artifact.txt @@ -0,0 +1 @@ +This is a test artifact. \ No newline at end of file diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java new file mode 100644 index 0000000000..f6baf3530c --- /dev/null +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java @@ -0,0 +1,78 @@ +package com.google.cloud.teleport.v2.testing; + +import static com.google.common.truth.Truth.assertThat; + +import org.junit.After; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public final class TestPropertiesTest { + private static final String ACCESS_TOKEN = "some-token"; + private static final String ARTIFACT_BUCKET = "test-bucket"; + private static final String PROJECT = "test-project"; + private static final String SPEC_PATH = "gs://test-bucket/some/spec/path"; + + private final TestProperties properties = new TestProperties(); + + @After + public void tearDown() { + System.clearProperty(TestProperties.ACCESS_TOKEN_KEY); + System.clearProperty(TestProperties.ARTIFACT_BUCKET_KEY); + System.clearProperty(TestProperties.PROJECT_KEY); + System.clearProperty(TestProperties.SPEC_PATH_KEY); + } + + @Test + public void testAllPropertiesSet() { + System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); + System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); + System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); + + assertThat(properties.accessToken()).isEqualTo(ACCESS_TOKEN); + assertThat(properties.artifactBucket()).isEqualTo(ARTIFACT_BUCKET); + assertThat(properties.project()).isEqualTo(PROJECT); + assertThat(properties.specPath()).isEqualTo(SPEC_PATH); + } + + @Test + public void testAccessTokenNotSet() { + System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); + System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); + + assertThat(properties.accessToken()).isNull(); + assertThat(properties.artifactBucket()).isEqualTo(ARTIFACT_BUCKET); + assertThat(properties.project()).isEqualTo(PROJECT); + assertThat(properties.specPath()).isEqualTo(SPEC_PATH); + } + + @Test(expected = IllegalStateException.class) + public void testArtifactBucketNotSet() { + System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); + System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); + + properties.artifactBucket(); + } + + @Test(expected = IllegalStateException.class) + public void testProjectNotSet() { + System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); + System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); + System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); + + properties.project(); + } + + @Test(expected = IllegalStateException.class) + public void testSpecPathNotSet() { + System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); + System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); + System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + + properties.specPath(); + } +} \ No newline at end of file diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java new file mode 100644 index 0000000000..3886f84a22 --- /dev/null +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java @@ -0,0 +1,297 @@ +package com.google.cloud.teleport.v2.testing.artifacts; + +import static com.google.common.truth.Truth.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyIterable; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.api.gax.paging.Page; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.Storage.BlobListOption; +import com.google.cloud.storage.Storage.BucketListOption; +import com.google.common.io.Resources; +import com.google.re2j.Pattern; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +@RunWith(JUnit4.class) +public final class ArtifactGcsSdkClientTest { + @Rule public final MockitoRule mockito = MockitoJUnit.rule(); + + @Mock private Storage client; + @Mock private Blob blob; + + private static final String LOCAL_PATH; + private static final byte[] TEST_ARTIFACT_CONTENTS; + static { + LOCAL_PATH = Resources.getResource("test-artifact.txt").getPath(); + try { + TEST_ARTIFACT_CONTENTS = Files.readAllBytes(Paths.get(LOCAL_PATH)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static final String BUCKET = "test-bucket"; + private static final String DIR_PATH = "some/dir/path"; + + @Captor private ArgumentCaptor bucketCaptor; + @Captor private ArgumentCaptor blobInfoCaptor; + @Captor private ArgumentCaptor contentsCaptor; + @Captor private ArgumentCaptor listOptionsCaptor; + @Captor private ArgumentCaptor> blobIdCaptor; + + @Test + public void testUploadArtifact() throws IOException { + when(client.create(any(BlobInfo.class), any(byte[].class))).thenReturn(blob); + + Blob actual = new ArtifactGcsSdkClient(client).uploadArtifact(BUCKET, DIR_PATH, LOCAL_PATH); + + verify(client).create(blobInfoCaptor.capture(), contentsCaptor.capture()); + + BlobInfo actualInfo = blobInfoCaptor.getValue(); + byte[] actualContents = contentsCaptor.getValue(); + + assertThat(actual).isSameInstanceAs(blob); + assertThat(actualInfo.getBucket()).isEqualTo(BUCKET); + assertThat(actualInfo.getName()).isEqualTo(DIR_PATH); + assertThat(actualContents).isEqualTo(TEST_ARTIFACT_CONTENTS); + } + + @Test(expected = IOException.class) + public void testUploadArtifactInvalidLocalPath() throws IOException { + when(client.create(any(BlobInfo.class), any())).thenReturn(blob); + new ArtifactGcsSdkClient(client).uploadArtifact( + BUCKET, DIR_PATH, "/some/invalid/path/please/do/not/make/this/file/valid"); + } + + @Test + public void testListArtifactsSinglePage() { + // Arrange + String name1 = DIR_PATH + "/blob1"; + String name2 = DIR_PATH + "/blob2"; + String name3 = DIR_PATH + "/blob3"; + ImmutableList page1 = ImmutableList.of( + mock(Blob.class), mock(Blob.class), mock(Blob.class)); + when(page1.get(0).getName()).thenReturn(name1); + when(page1.get(1).getName()).thenReturn(name2); + when(page1.get(2).getName()).thenReturn(name3); + + TestBlobPage allPages = createPages(page1); + when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); + + Pattern pattern = Pattern.compile(".*blob[13].*"); + + // Act + List actual = new ArtifactGcsSdkClient(client).listArtifacts(BUCKET, DIR_PATH, pattern); + + // Assert + verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); + + String actualBucket = bucketCaptor.getValue(); + BlobListOption actualOptions = listOptionsCaptor.getValue(); + + assertThat(actual).hasSize(2); + assertThat(actual.get(0).getName()).isEqualTo(name1); + assertThat(actual.get(1).getName()).isEqualTo(name3); + assertThat(actualBucket).isEqualTo(BUCKET); + assertThat(actualOptions).isEqualTo(BucketListOption.prefix(DIR_PATH)); + } + + @Test + public void testListArtifactsMultiplePages() { + // Arrange + String name1 = DIR_PATH + "/blob1"; + String name2 = DIR_PATH + "/blob2"; + String name3 = DIR_PATH + "/blob3"; + ImmutableList page1 = ImmutableList.of( + mock(Blob.class), mock(Blob.class)); + ImmutableList page2 = ImmutableList.of(mock(Blob.class)); + when(page1.get(0).getName()).thenReturn(name1); + when(page1.get(1).getName()).thenReturn(name2); + when(page2.get(0).getName()).thenReturn(name3); + + TestBlobPage allPages = createPages(page1, page2); + when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); + + Pattern pattern = Pattern.compile(".*blob[13].*"); + + // Act + List actual = new ArtifactGcsSdkClient(client).listArtifacts(BUCKET, DIR_PATH, pattern); + + // Assert + verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); + + String actualBucket = bucketCaptor.getValue(); + BlobListOption actualOptions = listOptionsCaptor.getValue(); + + assertThat(actual).hasSize(2); + assertThat(actual.get(0).getName()).isEqualTo(name1); + assertThat(actual.get(1).getName()).isEqualTo(name3); + assertThat(actualBucket).isEqualTo(BUCKET); + assertThat(actualOptions).isEqualTo(BucketListOption.prefix(DIR_PATH)); + } + + @Test + public void testListArtifactsNoArtifacts() { + TestBlobPage allPages = createPages(ImmutableList.of()); + when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); + Pattern pattern = Pattern.compile(".*blob[13].*"); + + List actual = new ArtifactGcsSdkClient(client).listArtifacts(BUCKET, DIR_PATH, pattern); + + verify(client).list(anyString(), any(BlobListOption.class)); + assertThat(actual).isEmpty(); + } + + @Test + public void testDeleteArtifactsSinglePage() { + // Arrange + BlobId id1 = BlobId.of(BUCKET, DIR_PATH + "/blob1"); + BlobId id2 = BlobId.of(BUCKET, DIR_PATH + "/blob2"); + BlobId id3 = BlobId.of(BUCKET, DIR_PATH + "/blob3"); + ImmutableList page1 = ImmutableList.of( + mock(Blob.class), mock(Blob.class), mock(Blob.class)); + when(page1.get(0).getBlobId()).thenReturn(id1); + when(page1.get(1).getBlobId()).thenReturn(id2); + when(page1.get(2).getBlobId()).thenReturn(id3); + + TestBlobPage allPages = createPages(page1); + when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); + + when(client.delete(anyIterable())).thenReturn(ImmutableList.of(true, false, true)); + + // Act + new ArtifactGcsSdkClient(client).deleteTestDir(BUCKET, DIR_PATH); + + // Assert + verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); + verify(client).delete(blobIdCaptor.capture()); + + String actualBucket = bucketCaptor.getValue(); + BlobListOption actualOption = listOptionsCaptor.getValue(); + Iterable actualIds = blobIdCaptor.getValue(); + + assertThat(actualBucket).isEqualTo(BUCKET); + assertThat(actualOption).isEqualTo(BucketListOption.prefix(DIR_PATH)); + assertThat(actualIds).containsExactly(id1, id2, id3); + } + + @Test + public void testDeleteArtifactsMultiplePages() { + // Arrange + BlobId id1 = BlobId.of(BUCKET, DIR_PATH + "/blob1"); + BlobId id2 = BlobId.of(BUCKET, DIR_PATH + "/blob2"); + BlobId id3 = BlobId.of(BUCKET, DIR_PATH + "/blob3"); + ImmutableList page1 = ImmutableList.of(mock(Blob.class), mock(Blob.class)); + ImmutableList page2 = ImmutableList.of(mock(Blob.class)); + when(page1.get(0).getBlobId()).thenReturn(id1); + when(page1.get(1).getBlobId()).thenReturn(id2); + when(page2.get(0).getBlobId()).thenReturn(id3); + + TestBlobPage allPages = createPages(page1, page2); + when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); + + // Technically, the second value is not realistic for the second call to delete, but it + // shouldn't mess anything up + when(client.delete(anyIterable())).thenReturn(ImmutableList.of(true, false)); + + // Act + new ArtifactGcsSdkClient(client).deleteTestDir(BUCKET, DIR_PATH); + + // Assert + verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); + verify(client, times(2)).delete(blobIdCaptor.capture()); + + String actualBucket = bucketCaptor.getValue(); + BlobListOption actualOption = listOptionsCaptor.getValue(); + List> actualBlobIds = blobIdCaptor.getAllValues(); + + assertThat(actualBucket).isEqualTo(BUCKET); + assertThat(actualOption).isEqualTo(BucketListOption.prefix(DIR_PATH)); + assertThat(actualBlobIds.get(0)).containsExactly(id1, id2); + assertThat(actualBlobIds.get(1)).containsExactly(id3); + } + + @Test + public void testDeleteArtifactsNoArtifacts() { + TestBlobPage allPages = createPages(ImmutableList.of()); + when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); + + new ArtifactGcsSdkClient(client).deleteTestDir(BUCKET, DIR_PATH); + + verify(client, never()).delete(anyIterable()); + } + + private static TestBlobPage createPages(ImmutableList... pageContents) { + if (pageContents.length == 0) { + return new TestBlobPage(ImmutableList.of()); + } + TestBlobPage first = new TestBlobPage(pageContents[0]); + TestBlobPage current = first; + for (int i = 1; i < pageContents.length; ++i) { + current.setNext(pageContents[i]); + current = current.next; + } + return first; + } + + private static final class TestBlobPage implements Page { + private TestBlobPage next; + private final ImmutableList contents; + + public TestBlobPage(ImmutableList contents) { + this.contents = contents; + this.next = null; + } + + public void setNext(ImmutableList contents) { + next = new TestBlobPage(contents); + } + + @Override + public boolean hasNextPage() { + return next != null; + } + + @Override + public String getNextPageToken() { + return "token"; + } + + @Override + public Page getNextPage() { + return next; + } + + @Override + public Iterable iterateAll() { + return contents; + } + + @Override + public Iterable getValues() { + return contents; + } + } +} \ No newline at end of file diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java new file mode 100644 index 0000000000..3556820f71 --- /dev/null +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java @@ -0,0 +1,37 @@ +package com.google.cloud.teleport.v2.testing.artifacts; + +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestDirName; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestDirPath; +import static com.google.common.truth.Truth.assertThat; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public final class ArtifactUtilsTest { + + // Not matching exact date, since it may fail if the test runs close enough to the change of + // date. + private static final String TEST_DIR_REGEX = + "\\d{8}-[a-fA-F0-9]{8}-([a-fA-F0-9]{4}-){3}[a-fA-F0-9]{12}"; + + private static final String BUCKET = "test-bucket"; + private static final String TEST_SUITE_DIR = "artifact-utils-test"; + + @Test + public void testCreateTestDirName() { + assertThat(createTestDirName()).matches(TEST_DIR_REGEX); + } + + @Test + public void testCreateTestDirPath() { + String name = "test-create-test-dir-path"; + + String actual = createTestDirPath(BUCKET, TEST_SUITE_DIR, name); + + String expectedPattern = String.format("gs://%s/%s/%s/%s", BUCKET, TEST_SUITE_DIR, name, TEST_DIR_REGEX); + + assertThat(actual).matches(expectedPattern); + } +} \ No newline at end of file diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java new file mode 100644 index 0000000000..b3d815e9fb --- /dev/null +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java @@ -0,0 +1,177 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import static com.google.common.truth.Truth.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation.Config; +import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation.Result; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; +import java.io.IOException; +import java.time.Duration; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +@RunWith(JUnit4.class) +public final class DataflowOperationTest { + @Rule public final MockitoRule mockito = MockitoJUnit.rule(); + + @Mock FlexTemplateClient client; + + private static final String PROJECT = "test-project"; + private static final String JOB_ID = "test-job-id"; + private static final Duration CHECK_AFTER = Duration.ofMillis(10); + private static final Duration TIMEOUT_AFTER = Duration.ofMillis(100); + + private static final Config DEFAULT_CONFIG = Config.builder() + .setProject(PROJECT) + .setJobId(JOB_ID) + .setCheckAfter(CHECK_AFTER) + .setTimeoutAfter(TIMEOUT_AFTER) + .build(); + + @Captor public ArgumentCaptor projectCaptor; + @Captor public ArgumentCaptor jobIdCaptor; + + @Test + public void testWaitUntilDone() throws IOException { + // Arrange + when(client.getJobStatus(any(), any())) + .thenReturn(JobState.QUEUED) + .thenReturn(JobState.RUNNING) + .thenReturn(JobState.CANCELLING) + .thenReturn(JobState.CANCELLED); + + // Act + Result result = DataflowOperation.waitUntilDone(client, DEFAULT_CONFIG); + + // Assert + verify(client, times(4)) + .getJobStatus(projectCaptor.capture(), jobIdCaptor.capture()); + + Set allProjects = new HashSet<>(projectCaptor.getAllValues()); + Set allJobIds = new HashSet<>(jobIdCaptor.getAllValues()); + + assertThat(allProjects).containsExactly(PROJECT); + assertThat(allJobIds).containsExactly(JOB_ID); + assertThat(result).isEqualTo(Result.JOB_FINISHED); + } + + @Test + public void testWaitUntilDoneTimeout() throws IOException { + when(client.getJobStatus(any(), any())).thenReturn(JobState.RUNNING); + Result result = DataflowOperation.waitUntilDone(client, DEFAULT_CONFIG); + assertThat(result).isEqualTo(Result.TIMEOUT); + } + + @Test + public void testWaitForCondition() throws IOException { + AtomicInteger callCount = new AtomicInteger(); + int totalCalls = 3; + Supplier checker = () -> callCount.incrementAndGet() >= totalCalls; + when(client.getJobStatus(any(), any())) + .thenReturn(JobState.RUNNING) + .thenThrow(new IOException()) + .thenReturn(JobState.RUNNING); + + Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, checker); + + verify(client, atMost(totalCalls)).getJobStatus(projectCaptor.capture(), jobIdCaptor.capture()); + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + assertThat(result).isEqualTo(Result.CONDITION_MET); + } + + @Test + public void testWaitForConditionJobFinished() throws IOException { + when(client.getJobStatus(any(), any())) + .thenReturn(JobState.RUNNING) + .thenReturn(JobState.CANCELLED); + + Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + + assertThat(result).isEqualTo(Result.JOB_FINISHED); + } + + @Test + public void testWaitForConditionTimeout() throws IOException { + when(client.getJobStatus(any(), any())).thenReturn(JobState.RUNNING); + + Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + + assertThat(result).isEqualTo(Result.TIMEOUT); + } + + @Test + public void testFinishAfterCondition() throws IOException { + // Arrange + AtomicInteger callCount = new AtomicInteger(); + int totalCalls = 3; + Supplier checker = () -> callCount.incrementAndGet() >= totalCalls; + + when(client.getJobStatus(any(), any())) + .thenReturn(JobState.RUNNING) + .thenThrow(new IOException()) + .thenReturn(JobState.RUNNING) + .thenReturn(JobState.CANCELLING) + .thenReturn(JobState.CANCELLED); + doAnswer(invocation -> null).when(client).cancelJob(any(), any()); + + // Act + Result result = DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, checker); + + // Assert + verify(client, atLeast(totalCalls)) + .getJobStatus(projectCaptor.capture(), jobIdCaptor.capture()); + verify(client).cancelJob(projectCaptor.capture(), jobIdCaptor.capture()); + + Set allProjects = new HashSet<>(projectCaptor.getAllValues()); + Set allJobIds = new HashSet<>(jobIdCaptor.getAllValues()); + + assertThat(allProjects).containsExactly(PROJECT); + assertThat(allJobIds).containsExactly(JOB_ID); + assertThat(result).isEqualTo(Result.CONDITION_MET); + } + + @Test + public void testFinishAfterConditionJobStopped() throws IOException { + when(client.getJobStatus(any(), any())) + .thenReturn(JobState.RUNNING) + .thenReturn(JobState.CANCELLED); + doAnswer(invocation -> null).when(client).cancelJob(projectCaptor.capture(), jobIdCaptor.capture()); + + Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + + verify(client, never()).cancelJob(any(), any()); + assertThat(result).isEqualTo(Result.JOB_FINISHED); + } + + @Test + public void testFinishAfterConditionTimeout() throws IOException { + when(client.getJobStatus(any(), any())) + .thenReturn(JobState.RUNNING); + doAnswer(invocation -> null).when(client).cancelJob(projectCaptor.capture(), jobIdCaptor.capture()); + + Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + + verify(client, never()).cancelJob(any(), any()); + assertThat(result).isEqualTo(Result.TIMEOUT); + } +} \ No newline at end of file diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java new file mode 100644 index 0000000000..50ed81aaac --- /dev/null +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java @@ -0,0 +1,18 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import static com.google.cloud.teleport.v2.testing.dataflow.DataflowUtils.createJobName; +import static com.google.common.truth.Truth.assertThat; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DataflowUtilsTest { + + @Test + public void testCreateJobName() { + String name = "create-job-name"; + assertThat(createJobName(name)).matches(name + "-\\d{14}"); + } +} \ No newline at end of file diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java new file mode 100644 index 0000000000..11d0bd725d --- /dev/null +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java @@ -0,0 +1,157 @@ +package com.google.cloud.teleport.v2.testing.dataflow; + +import static com.google.common.truth.Truth.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.Dataflow.Projects; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Update; +import com.google.api.services.dataflow.Dataflow.Projects.Locations.FlexTemplates; +import com.google.api.services.dataflow.Dataflow.Projects.Locations.FlexTemplates.Launch; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.LaunchFlexTemplateParameter; +import com.google.api.services.dataflow.model.LaunchFlexTemplateRequest; +import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse; +import com.google.auth.Credentials; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobInfo; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.Options; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Answers; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +@RunWith(JUnit4.class) +public final class FlexTemplateSdkClientTest{ + @Rule public final MockitoRule mockito = MockitoJUnit.rule(); + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private Dataflow client; + + private static final String PROJECT = "test-project"; + private static final String REGION = "us-east1"; + private static final String JOB_ID = "test-job-id"; + private static final String JOB_NAME = "test-job"; + private static final String SPEC_PATH = "gs://test-bucket/test-dir/test-spec.json"; + + private static final String PARAM_KEY = "key"; + private static final String PARAM_VALUE = "value"; + + @Captor private ArgumentCaptor projectCaptor; + @Captor private ArgumentCaptor regionCaptor; + @Captor private ArgumentCaptor jobIdCaptor; + @Captor private ArgumentCaptor requestCaptor; + @Captor private ArgumentCaptor jobCaptor; + + @Test + public void testCreateWithCredentials() { + Credentials credentials = mock(Credentials.class); + FlexTemplateSdkClient.builder().setCredentials(credentials).build(); + // Lack of exception is all we really can test + } + + @Test + public void testCreateWithNullCredentials() { + FlexTemplateSdkClient.builder().setCredentials(null).build(); + // Lack of exception is all we really can test + } + + @Test + public void testLaunchNewJob() throws IOException { + // Arrange + Launch launch = mock(Launch.class); + Job job = new Job().setId(JOB_ID).setCurrentState(JobState.QUEUED.toString()); + LaunchFlexTemplateResponse response = new LaunchFlexTemplateResponse().setJob(job); + + Options options = new Options(JOB_NAME, SPEC_PATH) + .setIsStreaming(true) + .addParameter(PARAM_KEY, PARAM_VALUE); + + when(getFlexTemplates(client).launch(projectCaptor.capture(), regionCaptor.capture(), requestCaptor.capture())).thenReturn(launch); + when(launch.execute()).thenReturn(response); + + // Act + JobInfo actual = FlexTemplateSdkClient.withDataflowClient(client).launchNewJob(PROJECT, REGION, options); + + // Assert + JobInfo expected = JobInfo.builder() + .setJobId(JOB_ID) + .setState(JobState.QUEUED) + .build(); + + LaunchFlexTemplateRequest expectedRequest = new LaunchFlexTemplateRequest() + .setLaunchParameter(new LaunchFlexTemplateParameter() + .setJobName(JOB_NAME) + .setContainerSpecGcsPath(SPEC_PATH) + .setParameters(ImmutableMap.of(PARAM_KEY, PARAM_VALUE, "isStreaming", "true"))); + + assertThat(actual).isEqualTo(expected); + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); + assertThat(requestCaptor.getValue()).isEqualTo(expectedRequest); + } + + @Test(expected = IOException.class) + public void testLaunchNewJobThrowsException() throws IOException { + when(getFlexTemplates(client).launch(any(), any(), any())).thenThrow(new IOException()); + FlexTemplateSdkClient.withDataflowClient(client).launchNewJob(PROJECT, REGION, new Options(JOB_NAME, SPEC_PATH)); + } + + @Test + public void testGetJobStatus() throws IOException { + Get get = mock(Get.class); + Job job = new Job().setCurrentState(JobState.RUNNING.toString()); + when(getProjectJobs(client).get(projectCaptor.capture(), jobIdCaptor.capture())).thenReturn(get); + when(get.execute()).thenReturn(job); + + JobState actual = FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, JOB_ID); + + assertThat(actual).isEqualTo(JobState.RUNNING); + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + } + + @Test(expected = IOException.class) + public void testGetJobThrowsException() throws IOException { + when(getProjectJobs(client).get(any(), any())).thenThrow(new IOException()); + FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, JOB_ID); + } + + @Test + public void testCancelJob() throws IOException { + Update update = mock(Update.class); + when(getProjectJobs(client).update(projectCaptor.capture(), jobIdCaptor.capture(), + jobCaptor.capture())).thenReturn(update); + when(update.execute()).thenReturn(new Job()); + + FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, JOB_ID); + + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + assertThat(jobCaptor.getValue().getCurrentState()).isEqualTo(JobState.CANCELLED.toString()); + } + + @Test(expected = IOException.class) + public void testCancelJobThrowsException() throws IOException { + when(getProjectJobs(client).update(any(), any(), any())).thenThrow(new IOException()); + FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, JOB_ID); + } + + private static Projects.Jobs getProjectJobs(Dataflow client) { + return client.projects().jobs(); + } + + private static FlexTemplates getFlexTemplates(Dataflow client) { + return client.projects().locations().flexTemplates(); + } +} \ No newline at end of file From 49b6092b13b96fb6bf6424180a811d95be762b45 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 26 Jan 2022 09:52:30 -0500 Subject: [PATCH 069/145] Add test for streaming generated data to GCS --- v2/pom.xml | 3 + v2/streaming-data-generator/pom.xml | 10 +- .../templates/StreamingDataGeneratorIT.java | 114 ++++++++++++--- .../src/test/resources/gameevent.json | 11 ++ v2/testing/pom.xml | 6 - .../teleport/v2/testing/TestProperties.java | 47 ++++++- .../v2/testing/artifacts/ArtifactClient.java | 39 ++++++ .../artifacts/ArtifactGcsSdkClient.java | 80 +++++++---- .../v2/testing/artifacts/ArtifactUtils.java | 61 +++++++- .../v2/testing/artifacts/package-info.java | 3 +- .../testing/dataflow/DataflowOperation.java | 128 +++++++++++++---- .../v2/testing/dataflow/DataflowUtils.java | 33 ++++- .../testing/dataflow/FlexTemplateClient.java | 132 +++++++++++++++--- .../dataflow/FlexTemplateSdkClient.java | 105 +++++++++----- .../v2/testing/dataflow/package-info.java | 3 +- .../teleport/v2/testing/package-info.java | 2 + .../v2/testing/TestPropertiesTest.java | 43 +++++- .../artifacts/ArtifactGcsSdkClientTest.java | 34 +++-- .../testing/artifacts/ArtifactUtilsTest.java | 47 +++++-- .../dataflow/DataflowOperationTest.java | 91 ++++++++---- .../testing/dataflow/DataflowUtilsTest.java | 18 ++- .../dataflow/FlexTemplateSdkClientTest.java | 125 +++++++++++------ 22 files changed, 880 insertions(+), 255 deletions(-) create mode 100644 v2/streaming-data-generator/src/test/resources/gameevent.json create mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java diff --git a/v2/pom.xml b/v2/pom.xml index f3a822974f..6d927cbd8b 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -572,6 +572,9 @@ --no-deprecation-warning + + true ${excluded.spanner.tests} diff --git a/v2/streaming-data-generator/pom.xml b/v2/streaming-data-generator/pom.xml index c73de3fcd5..3d1e0769cd 100644 --- a/v2/streaming-data-generator/pom.xml +++ b/v2/streaming-data-generator/pom.xml @@ -32,6 +32,7 @@ 1.10 1.23 + 1.0.1 @@ -62,13 +63,20 @@ snakeyaml ${snakeyaml.version} + + + + com.google.truth + truth + ${truth.version} + test + com.google.cloud.teleport.v2 testing 1.0-SNAPSHOT test - diff --git a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java index fe2a21d520..444ac4bf9f 100644 --- a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java +++ b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java @@ -1,21 +1,60 @@ +/* + * Copyright (C) 2022 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.v2.templates; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createGcsClient; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestPath; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestSuiteDirPath; +import static com.google.cloud.teleport.v2.testing.dataflow.DataflowUtils.createJobName; +import static com.google.common.truth.Truth.assertThat; + +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.Storage; import com.google.cloud.teleport.v2.templates.StreamingDataGenerator.SinkType; +import com.google.cloud.teleport.v2.testing.TestProperties; +import com.google.cloud.teleport.v2.testing.artifacts.ArtifactClient; +import com.google.cloud.teleport.v2.testing.artifacts.ArtifactGcsSdkClient; import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation; +import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation.Result; import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient; import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobInfo; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.Options; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.LaunchOptions; import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateSdkClient; +import com.google.common.io.Resources; +import com.google.re2j.Pattern; import java.io.IOException; -import java.time.Duration; +import java.util.List; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +/** Integration test for {@link StreamingDataGenerator}. */ @RunWith(JUnit4.class) public final class StreamingDataGeneratorIT { + private static final TestProperties PROPERTIES = new TestProperties(); + + private static final String SCHEMA_FILE = "gameevent.json"; + private static final String LOCAL_SCHEMA_PATH = Resources.getResource(SCHEMA_FILE).getPath(); - private static final Duration MAX_WAIT_TIME = Duration.ofMinutes(15); + private static final String TEST_ROOT_DIR = "streaming-data-generator"; + private static final String TEST_DIR = createTestSuiteDirPath(TEST_ROOT_DIR); + private static final String SCHEMA_FILE_GCS_PATH = String.format("%s/%s", TEST_DIR, SCHEMA_FILE); private static final String NUM_SHARDS_KEY = "numShards"; private static final String OUTPUT_DIRECTORY_KEY = "outputDirectory"; @@ -27,20 +66,63 @@ public final class StreamingDataGeneratorIT { private static final String DEFAULT_QPS = "15"; private static final String DEFAULT_WINDOW_DURATION = "60s"; + private static ArtifactClient artifactClient; + + @BeforeClass + public static void setUpClass() throws IOException { + Storage gcsClient = createGcsClient(PROPERTIES.googleCredentials()); + artifactClient = new ArtifactGcsSdkClient(gcsClient); + artifactClient.uploadArtifact( + PROPERTIES.artifactBucket(), SCHEMA_FILE_GCS_PATH, LOCAL_SCHEMA_PATH); + } + + @AfterClass + public static void tearDownClass() { + artifactClient.deleteTestDir(PROPERTIES.artifactBucket(), TEST_DIR); + } + @Test public void testFakeMessagesToGcs() throws IOException { - String name = "teleport_flex_streaming_data_generator_gcs"; - Options options = new Options(name, "TODO") - .setIsStreaming(true) - .addParameter(SCHEMA_LOCATION_KEY, "TODO") - .addParameter(QPS_KEY, DEFAULT_QPS) - .addParameter(SINK_TYPE_KEY, SinkType.GCS.name()) - .addParameter(WINDOW_DURATION_KEY, DEFAULT_WINDOW_DURATION) - .addParameter(OUTPUT_DIRECTORY_KEY, "TODO") - .addParameter(NUM_SHARDS_KEY, "1"); - FlexTemplateClient dataflow = FlexTemplateSdkClient.builder().build(); - - JobInfo info = dataflow.launchNewJob("TODO", "TODO", options); - DataflowOperation.waitForConditionAndFinish(dataflow, null, () -> false); + String name = "teleport-flex-streaming-data-generator-gcs"; + String outputDir = createTestPath(TEST_DIR, name); + String jobName = createJobName(name); + LaunchOptions options = + LaunchOptions.builder(jobName, PROPERTIES.specPath()) + .addParameter( + SCHEMA_LOCATION_KEY, + String.format("gs://%s/%s", PROPERTIES.artifactBucket(), SCHEMA_FILE_GCS_PATH)) + .addParameter(QPS_KEY, DEFAULT_QPS) + .addParameter(SINK_TYPE_KEY, SinkType.GCS.name()) + .addParameter(WINDOW_DURATION_KEY, DEFAULT_WINDOW_DURATION) + .addParameter( + OUTPUT_DIRECTORY_KEY, + String.format("gs://%s/%s", PROPERTIES.artifactBucket(), outputDir)) + .addParameter(NUM_SHARDS_KEY, "1") + .build(); + FlexTemplateClient dataflow = + FlexTemplateSdkClient.builder().setCredentials(PROPERTIES.googleCredentials()).build(); + + JobInfo info = dataflow.launchNewJob(PROPERTIES.project(), PROPERTIES.region(), options); + assertThat(info.state()).isIn(JobState.RUNNING_STATES); + + Result result = + DataflowOperation.waitForConditionAndFinish( + dataflow, + createConfig(info), + () -> { + List outputFiles = + artifactClient.listArtifacts( + PROPERTIES.artifactBucket(), outputDir, Pattern.compile(".*output-.*")); + return !outputFiles.isEmpty(); + }); + assertThat(result).isEqualTo(Result.CONDITION_MET); + } + + private static DataflowOperation.Config createConfig(JobInfo info) { + return DataflowOperation.Config.builder() + .setJobId(info.jobId()) + .setProject(PROPERTIES.project()) + .setRegion(PROPERTIES.region()) + .build(); } } diff --git a/v2/streaming-data-generator/src/test/resources/gameevent.json b/v2/streaming-data-generator/src/test/resources/gameevent.json new file mode 100644 index 0000000000..fcf52e69af --- /dev/null +++ b/v2/streaming-data-generator/src/test/resources/gameevent.json @@ -0,0 +1,11 @@ +{ + "eventId": "{{uuid()}}", + "eventTime": {{timestamp()}}, + "ipv4": "{{ipv4()}}", + "ipv6": "{{ipv6()}}", + "country": "{{country()}}", + "username": "{{username()}}", + "quest": "{{random("A Break In the Ice", "Ghosts of Perdition", "Survive the Low Road")}}", + "score": {{integer(100, 10000)}}, + "completed": {{bool()}} +} \ No newline at end of file diff --git a/v2/testing/pom.xml b/v2/testing/pom.xml index 9ea9d2ef61..e15fedb47f 100644 --- a/v2/testing/pom.xml +++ b/v2/testing/pom.xml @@ -12,16 +12,10 @@ - 2.5.0 1.0.1 - - com.google.cloud.dataflow - google-cloud-dataflow-java-sdk-all - ${dataflow-client.version} - com.google.cloud google-cloud-storage diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java index 01baaeb503..6d6a10e57e 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java @@ -1,11 +1,28 @@ +/* + * Copyright (C) 2022 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.v2.testing; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import com.google.auth.Credentials; +import com.google.auth.oauth2.AccessToken; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.common.base.Strings; import java.util.HashMap; import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Utility for accessing system properties set for the test. @@ -14,16 +31,18 @@ * `-Dproject=my-project`. */ public final class TestProperties { - private static final Logger LOG = LoggerFactory.getLogger(TestProperties.class); - public static final String ACCESS_TOKEN_KEY = "accessToken"; public static final String ARTIFACT_BUCKET_KEY = "artifactBucket"; public static final String PROJECT_KEY = "project"; + public static final String REGION_KEY = "region"; public static final String SPEC_PATH_KEY = "specPath"; + public static final String DEFAULT_REGION = "us-central1"; + private static String accessToken; private static String artifactBucket; private static String project; + private static String region; private static String specPath; private final Map initialized; @@ -33,21 +52,27 @@ public TestProperties() { initialized.put(ACCESS_TOKEN_KEY, false); initialized.put(ARTIFACT_BUCKET_KEY, false); initialized.put(PROJECT_KEY, false); + initialized.put(REGION_KEY, false); initialized.put(SPEC_PATH_KEY, false); } public String accessToken() { if (!initialized.get(ACCESS_TOKEN_KEY)) { accessToken = System.getProperty(ACCESS_TOKEN_KEY, null); + checkState(!Strings.isNullOrEmpty(accessToken), "%s is required", ACCESS_TOKEN_KEY); initialized.replace(ACCESS_TOKEN_KEY, true); } return accessToken; } + public Credentials googleCredentials() { + return new GoogleCredentials(new AccessToken(accessToken(), /* expirationTime= */ null)); + } + public String artifactBucket() { if (!initialized.get(ARTIFACT_BUCKET_KEY)) { artifactBucket = System.getProperty(ARTIFACT_BUCKET_KEY, null); - checkState(artifactBucket != null, "%s is required", ARTIFACT_BUCKET_KEY); + checkState(!Strings.isNullOrEmpty(artifactBucket), "%s is required", ARTIFACT_BUCKET_KEY); initialized.replace(ARTIFACT_BUCKET_KEY, true); } return artifactBucket; @@ -56,16 +81,24 @@ public String artifactBucket() { public String project() { if (!initialized.get(PROJECT_KEY)) { project = System.getProperty(PROJECT_KEY, null); - checkState(project != null, "%s is required", PROJECT_KEY); + checkState(!Strings.isNullOrEmpty(project), "%s is required", PROJECT_KEY); initialized.replace(PROJECT_KEY, true); } return project; } + public String region() { + if (!initialized.get(REGION_KEY)) { + region = System.getProperty(REGION_KEY, DEFAULT_REGION); + initialized.replace(REGION_KEY, true); + } + return region; + } + public String specPath() { if (!initialized.get(SPEC_PATH_KEY)) { specPath = System.getProperty(SPEC_PATH_KEY, null); - checkState(specPath != null, "%s is required", SPEC_PATH_KEY); + checkState(!Strings.isNullOrEmpty(specPath), "%s is required", SPEC_PATH_KEY); initialized.replace(SPEC_PATH_KEY, true); } return specPath; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java index bb4932948d..672e7685e1 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java @@ -1,3 +1,18 @@ +/* + * Copyright (C) 2022 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.v2.testing.artifacts; import com.google.cloud.storage.Blob; @@ -5,10 +20,34 @@ import java.io.IOException; import java.util.List; +/** Interface for working with test artifacts. */ public interface ArtifactClient { + /** + * Uploads a local file to GCS. + * + * @param bucket the GCS bucket to upload to + * @param gcsPath the path from the bucket root to upload to + * @param localPath the path to the local file + * @return the {@link Blob} that was created + * @throws IOException if the local file cannot be read + */ Blob uploadArtifact(String bucket, String gcsPath, String localPath) throws IOException; + /** + * Lists all artifacts in the given directory that match a given regex. + * + * @param bucket the bucket the artifacts are in + * @param testDirPath the directory in the bucket that the artifacts are in + * @param regex the regex to use for matching artifacts + * @return all the {@link Blob}s that match the regex + */ List listArtifacts(String bucket, String testDirPath, Pattern regex); + /** + * Removes the directory from the bucket. + * + * @param bucket the bucket with the directory to remove + * @param testDirPath the directory to remove + */ void deleteTestDir(String bucket, String testDirPath); } diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java index 661eaa7bd3..cdd0f028b1 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java @@ -1,3 +1,18 @@ +/* + * Copyright (C) 2022 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.v2.testing.artifacts; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList.toImmutableList; @@ -30,8 +45,9 @@ public ArtifactGcsSdkClient(Storage client) { this.client = client; } - public Blob uploadArtifact(String bucket, String gcsPath, String localPath) - throws IOException { + @Override + public Blob uploadArtifact(String bucket, String gcsPath, String localPath) throws IOException { + LOG.info("Uploading {} to {} under {}", localPath, gcsPath, bucket); BlobId id = BlobId.of(bucket, gcsPath); BlobInfo info = BlobInfo.newBuilder(id).build(); @@ -40,38 +56,50 @@ public Blob uploadArtifact(String bucket, String gcsPath, String localPath) return client.create(info, contents); } + @Override public List listArtifacts(String bucket, String testDirPath, Pattern regex) { List result = new ArrayList<>(); - consumeTestDir(bucket, testDirPath, blobs -> { - for (Blob blob : blobs) { - if (regex.matches(blob.getName())) { - result.add(blob); - } - } - }); + consumeTestDir( + bucket, + testDirPath, + blobs -> { + for (Blob blob : blobs) { + if (regex.matches(blob.getName())) { + result.add(blob); + } + } + }); return result; } + @Override public void deleteTestDir(String bucket, String testDirPath) { - consumeTestDir(bucket, testDirPath, blobs -> { - // Go through the Iterable overload, since the other ones make it very difficult to - // do thorough testing with Mockito - ImmutableList blobIds = StreamSupport.stream(blobs.spliterator(), false) - .map(Blob::getBlobId) - .collect(toImmutableList()); - if (blobIds.isEmpty()) { - return; - } - List deleted = client.delete(blobIds); - for (int i = 0; i < blobIds.size(); ++i) { - if (!deleted.get(i)) { - LOG.warn("Blob {} not deleted", blobIds.get(i).getName()); - } - } - }); + LOG.info("Deleting everything in {} under {}", testDirPath, bucket); + consumeTestDir( + bucket, + testDirPath, + blobs -> { + // Go through the Iterable overload, since the other ones make it very difficult + // to + // do thorough testing with Mockito + ImmutableList blobIds = + StreamSupport.stream(blobs.spliterator(), false) + .map(Blob::getBlobId) + .collect(toImmutableList()); + if (blobIds.isEmpty()) { + return; + } + List deleted = client.delete(blobIds); + for (int i = 0; i < blobIds.size(); ++i) { + if (!deleted.get(i)) { + LOG.warn("Blob {} not deleted", blobIds.get(i).getName()); + } + } + }); } - private void consumeTestDir(String bucket, String testDirPath, Consumer> consumeBlobs) { + private void consumeTestDir( + String bucket, String testDirPath, Consumer> consumeBlobs) { Page blobs = getFirstTestDirPage(bucket, testDirPath); while (true) { consumeBlobs.accept(blobs.getValues()); diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java index 766d97f44f..4af07e3813 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java @@ -1,21 +1,76 @@ +/* + * Copyright (C) 2022 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.v2.testing.artifacts; +import com.google.auth.Credentials; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; import java.time.Instant; import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.UUID; +import javax.annotation.Nullable; /** Utilities for working with test artifacts. */ public final class ArtifactUtils { private ArtifactUtils() {} + /** Creates a unique name for the test directory. */ public static String createTestDirName() { - return String.format("%s-%s", + return String.format( + "%s-%s", DateTimeFormatter.ofPattern("yyyyMMdd").withZone(ZoneId.of("UTC")).format(Instant.now()), UUID.randomUUID()); } - public static String createTestDirPath(String bucket, String suiteDir, String testDir) { - return String.format("gs://%s/%s/%s/%s", bucket, suiteDir, testDir, createTestDirName()); + /** + * Creates a unique path for the test suite. + * + * @param suiteDir the name of the test suite. This is generally the class with all the tests in + * it. + */ + public static String createTestSuiteDirPath(String suiteDir) { + return String.format("%s/%s", suiteDir, createTestDirName()); + } + + /** + * Creates a path for artifacts from an individual test to go into. + * + * @param suiteDirPath the name of the test suite. This is generally the class with all the tests + * in it. + * @param testName the name of the test. It is the responsibility of the caller to make sure all + * their test names are unique. + */ + public static String createTestPath(String suiteDirPath, String testName) { + return String.format("%s/%s", suiteDirPath, testName); + } + + /** + * Creates a client for GCS with the given credentials. + * + * @param credentials credentials to use for connecting. If not chosen, then this will use the + * system credentials. Using system credentials is intended only for local testing. Otherwise, + * it is best to pass in a short-lived access token. + * @return a {@link Storage} client for running GCS operations + */ + public static Storage createGcsClient(@Nullable Credentials credentials) { + StorageOptions.Builder builder = StorageOptions.newBuilder(); + if (credentials != null) { + builder.setCredentials(credentials); + } + return builder.build().getService(); } } diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java index 671b9c8243..7c70947d4a 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java @@ -1 +1,2 @@ -package com.google.cloud.teleport.v2.testing.artifacts; \ No newline at end of file +/** Package for working with test artifacts. */ +package com.google.cloud.teleport.v2.testing.artifacts; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java index 03ec4ed159..76b7aa44f5 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java @@ -1,5 +1,22 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; + import com.google.auto.value.AutoValue; import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; import java.io.IOException; @@ -11,49 +28,88 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** Utilities for managing Dataflow jobs. */ public final class DataflowOperation { private static final Logger LOG = LoggerFactory.getLogger(DataflowOperation.class); + /** The result of running an operation. */ public enum Result { CONDITION_MET, JOB_FINISHED, TIMEOUT } - public static final ImmutableSet DONE_STATES = ImmutableSet.of( - JobState.CANCELLED, - JobState.DONE, - JobState.DRAINED, - JobState.FAILED, - JobState.STOPPED); + public static final ImmutableSet DONE_STATES = + ImmutableSet.of( + JobState.CANCELLED, JobState.DONE, JobState.DRAINED, JobState.FAILED, JobState.STOPPED); - public static final ImmutableSet FINISHING_STATES = ImmutableSet.of( - JobState.DRAINING, - JobState.CANCELLING); + public static final ImmutableSet FINISHING_STATES = + ImmutableSet.of(JobState.DRAINING, JobState.CANCELLING); private DataflowOperation() {} + /** + * Waits until the given job is done, timing out it if runs for too long. + * + *

If the job is a batch job, it should complete eventually. If it is a streaming job, this + * will time out unless the job is explicitly cancelled or drained. + * + * @param client the {@link FlexTemplateClient} to use for performing operations + * @param config the configuration for performing the operation + * @return the result, which will be either {@link Result#JOB_FINISHED} or {@link Result#TIMEOUT} + */ public static Result waitUntilDone(FlexTemplateClient client, Config config) { - return finishOrTimeout(config, () -> false, () -> jobIsDone(client, config.project(), config.jobId())); + return finishOrTimeout( + config, + () -> false, + () -> jobIsDone(client, config.project(), config.region(), config.jobId())); } - public static Result waitForCondition(FlexTemplateClient client, Config config, Supplier conditionCheck) { - return finishOrTimeout(config, conditionCheck, () -> jobIsDoneOrFinishing(client, - config.project(), config.jobId())); + /** + * Waits until a given condition is met OR when the job enters a state that indicates that it is + * done or ready to be done. + * + * @param client the {@link FlexTemplateClient} to use for performing operations + * @param config the configuration for performing operations + * @param conditionCheck a {@link Supplier} that will be called periodically to check if the + * condition is met + * @return the result, which could be any value in {@link Result} + */ + public static Result waitForCondition( + FlexTemplateClient client, Config config, Supplier conditionCheck) { + return finishOrTimeout( + config, + conditionCheck, + () -> jobIsDoneOrFinishing(client, config.project(), config.region(), config.jobId())); } - public static Result waitForConditionAndFinish(FlexTemplateClient client, Config config, Supplier conditionCheck) + /** + * Waits until a given condition is met OR when a job enters a state that indicates that it is + * done or ready to be done. + * + *

If the condition was met before the job entered a done or finishing state, then this will + * cancel the job and wait for the job to enter a done state. + * + * @param client the {@link FlexTemplateClient} to use for performing operations + * @param config the configuration for performing operations + * @param conditionCheck a {@link Supplier} that will be called periodically to check if the + * condition is met + * @return the result of waiting for the condition, not of waiting for the job to be done + * @throws IOException if there is an issue cancelling the job + */ + public static Result waitForConditionAndFinish( + FlexTemplateClient client, Config config, Supplier conditionCheck) throws IOException { - Instant start = Instant.now(); Result conditionStatus = waitForCondition(client, config, conditionCheck); if (conditionStatus != Result.JOB_FINISHED) { - client.cancelJob(config.project(), config.jobId()); + client.cancelJob(config.project(), config.region(), config.jobId()); waitUntilDone(client, config); } return conditionStatus; } - private static Result finishOrTimeout(Config config, Supplier conditionCheck, Supplier stopChecking) { + private static Result finishOrTimeout( + Config config, Supplier conditionCheck, Supplier stopChecking) { Instant start = Instant.now(); LOG.info("Making initial finish check."); @@ -77,16 +133,18 @@ private static Result finishOrTimeout(Config config, Supplier condition if (stopChecking.get()) { return Result.JOB_FINISHED; } - LOG.info("Job not finished. Will check again in {} seconds", config.checkAfter().getSeconds()); + LOG.info( + "Job not finished. Will check again in {} seconds", config.checkAfter().getSeconds()); } LOG.warn("Neither the condition or job completion were fulfilled on time."); return Result.TIMEOUT; } - private static boolean jobIsDone(FlexTemplateClient client, String project, String jobId) { + private static boolean jobIsDone( + FlexTemplateClient client, String project, String region, String jobId) { try { - JobState state = client.getJobStatus(project, jobId); + JobState state = client.getJobStatus(project, region, jobId); LOG.info("Job is in state {}", state); return DONE_STATES.contains(state); } catch (IOException e) { @@ -95,9 +153,10 @@ private static boolean jobIsDone(FlexTemplateClient client, String project, Stri } } - private static boolean jobIsDoneOrFinishing(FlexTemplateClient client, String project, String jobId) { + private static boolean jobIsDoneOrFinishing( + FlexTemplateClient client, String project, String region, String jobId) { try { - JobState state = client.getJobStatus(project, jobId); + JobState state = client.getJobStatus(project, region, jobId); LOG.info("Job is in state {}", state); return DONE_STATES.contains(state) || FINISHING_STATES.contains(state); } catch (IOException e) { @@ -110,11 +169,17 @@ private static boolean timeIsLeft(Instant start, Duration maxWaitTime) { return Duration.between(start, Instant.now()).minus(maxWaitTime).isNegative(); } + /** Configuration for running an operation. */ @AutoValue - public static abstract class Config { + public abstract static class Config { public abstract String project(); + public abstract String jobId(); + + public abstract String region(); + public abstract Duration checkAfter(); + public abstract Duration timeoutAfter(); // TODO(zhoufek): Also let users set the maximum number of exceptions. @@ -124,23 +189,26 @@ public static Builder builder() { .setTimeoutAfter(Duration.ofMinutes(15)); } + /** Builder for a {@link Config}. */ @AutoValue.Builder - public static abstract class Builder { + public abstract static class Builder { public abstract Builder setProject(String value); + + public abstract Builder setRegion(String value); + public abstract Builder setJobId(String value); + public abstract Builder setCheckAfter(Duration value); + public abstract Builder setTimeoutAfter(Duration value); abstract Config autoBuild(); public Config build() { Config config = autoBuild(); - if (Strings.isNullOrEmpty(config.project())) { - throw new IllegalStateException("Project cannot be null or empty"); - } - if (Strings.isNullOrEmpty(config.jobId())) { - throw new IllegalStateException("Job ID cannot be null or empty"); - } + checkState(!Strings.isNullOrEmpty(config.project()), "Project must be set"); + checkState(!Strings.isNullOrEmpty(config.region()), "Region must be set"); + checkState(!Strings.isNullOrEmpty(config.jobId()), "Job id must be set"); return config; } } diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java index dfc8773dcd..152629ec63 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java @@ -1,15 +1,44 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; import java.time.Instant; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +/** Utilities to make working with Dataflow easier. */ public final class DataflowUtils { private DataflowUtils() {} + /** + * Creates a job name. + * + *

The job name will normally be unique, but this is not guaranteed if multiple jobs with the + * same prefix are requested in a short period of time. + * + * @param prefix a prefix for the job + * @return the prefix plus some way of identifying it separate from other jobs with the same + * prefix + */ public static String createJobName(String prefix) { - return String.format("%s-%s", + return String.format( + "%s-%s", prefix, - DateTimeFormatter.ofPattern("yyyyMMddHHmmss").withZone(ZoneId.of("UTC")).format(Instant.now())); + DateTimeFormatter.ofPattern("yyyyMMddHHmmss") + .withZone(ZoneId.of("UTC")) + .format(Instant.now())); } } diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java index 9aaa62f0e2..88fff78916 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java @@ -1,12 +1,31 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.util.HashMap; import java.util.Map; +import javax.annotation.Nullable; +/** Client for working with Flex templates. */ public interface FlexTemplateClient { + /** Enum representing known Dataflow job states. */ enum JobState { UNKNOWN("JOB_STATE_UNKNOWN"), STOPPED("JOB_STATE_STOPPED"), @@ -24,12 +43,20 @@ enum JobState { private static final String DATAFLOW_PREFIX = "JOB_STATE_"; + public static final ImmutableSet RUNNING_STATES = + ImmutableSet.of(RUNNING, UPDATED, PENDING, QUEUED); + private final String text; JobState(String text) { this.text = text; } + /** + * Parses the state from Dataflow. + * + *

Always use this in place of valueOf. + */ public static JobState parse(String fromDataflow) { return valueOf(fromDataflow.replace(DATAFLOW_PREFIX, "")); } @@ -40,24 +67,16 @@ public String toString() { } } - class Options { + /** LaunchOptions for starting a Dataflow job. */ + class LaunchOptions { private final String jobName; - private final Map parameters; + private final ImmutableMap parameters; private final String specPath; - public Options(String jobName, String specPath) { - this.jobName = jobName; - this.specPath = specPath; - this.parameters = new HashMap<>(); - } - - public Options addParameter(String key, String value) { - parameters.put(key, value); - return this; - } - - public Options setIsStreaming(boolean value) { - return addParameter("isStreaming", Boolean.toString(value)); + private LaunchOptions(Builder builder) { + this.jobName = builder.jobName; + this.parameters = ImmutableMap.copyOf(builder.parameters); + this.specPath = builder.specPath; } public String jobName() { @@ -65,14 +84,54 @@ public String jobName() { } public ImmutableMap parameters() { - return ImmutableMap.copyOf(parameters); + return parameters; } public String specPath() { return specPath; } + + public static Builder builder(String jobName, String specPath) { + return new Builder(jobName, specPath); + } + + /** Builder for the {@link LaunchOptions}. */ + public static final class Builder { + private final String jobName; + private final Map parameters; + private final String specPath; + + private Builder(String jobName, String specPath) { + this.jobName = jobName; + this.parameters = new HashMap<>(); + this.specPath = specPath; + } + + public String getJobName() { + return jobName; + } + + @Nullable + public String getParameter(String key) { + return parameters.get(key); + } + + public Builder addParameter(String key, String value) { + parameters.put(key, value); + return this; + } + + public String getSpecPath() { + return specPath; + } + + public LaunchOptions build() { + return new LaunchOptions(this); + } + } } + /** Info about the job from what Dataflow returned. */ @AutoValue abstract class JobInfo { public abstract String jobId(); @@ -83,17 +142,46 @@ public static Builder builder() { return new AutoValue_FlexTemplateClient_JobInfo.Builder(); } + /** Builder for {@link JobInfo}. */ @AutoValue.Builder - public static abstract class Builder { + public abstract static class Builder { public abstract Builder setJobId(String value); + public abstract Builder setState(JobState value); + public abstract JobInfo build(); } } - JobInfo launchNewJob(String project, String region, Options options) throws IOException; - - JobState getJobStatus(String project, String jobId) throws IOException; - - void cancelJob(String project, String jobId) throws IOException; + /** + * Launches a new job. + * + * @param project the project to run the job in + * @param region the region to run the job in (e.g. us-east1) + * @param options options for configuring the job + * @return info about the request to launch a new job + * @throws IOException if there is an issue sending the request + */ + JobInfo launchNewJob(String project, String region, LaunchOptions options) throws IOException; + + /** + * Gets the current status of a job. + * + * @param project the project that the job is running under + * @param region the region that the job was launched in + * @param jobId the id of the job + * @return the current state of the job + * @throws IOException if there is an issue sending the request + */ + JobState getJobStatus(String project, String region, String jobId) throws IOException; + + /** + * Cancels the given job. + * + * @param project the project that the job is running under + * @param region the region that the job was launched in + * @param jobId the id of the job to cancel + * @throws IOException if there is an issue sending the request + */ + void cancelJob(String project, String region, String jobId) throws IOException; } diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java index a92aa2bf9d..30df83118b 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java @@ -1,7 +1,21 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; import com.google.api.client.googleapis.util.Utils; -import com.google.api.client.http.HttpRequestInitializer; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.LaunchFlexTemplateParameter; @@ -9,30 +23,23 @@ import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse; import com.google.auth.Credentials; import com.google.auth.http.HttpCredentialsAdapter; +import com.google.common.base.Strings; import java.io.IOException; -import java.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Client for interacting with Dataflow Flex Templates using the Dataflow SDK. */ public final class FlexTemplateSdkClient implements FlexTemplateClient { + private static final Logger LOG = LoggerFactory.getLogger(FlexTemplateSdkClient.class); private final Dataflow client; - // TODO(zhoufek): Let users set this in options. - private static final Duration REQUEST_TIMEOUT = Duration.ofSeconds(30); - private FlexTemplateSdkClient(Builder builder) { - HttpRequestInitializer initializer = builder.getCredentials() != null - ? new HttpCredentialsAdapter(builder.getCredentials()) - : request -> { - request.setConnectTimeout((int) REQUEST_TIMEOUT.toMillis()); - request.setReadTimeout((int) REQUEST_TIMEOUT.toMillis()); - request.setWriteTimeout((int) REQUEST_TIMEOUT.toMillis()); - }; - - this.client = new Dataflow( - Utils.getDefaultTransport(), - Utils.getDefaultJsonFactory(), - initializer); + this.client = + new Dataflow( + Utils.getDefaultTransport(), + Utils.getDefaultJsonFactory(), + new HttpCredentialsAdapter(builder.getCredentials())); } private FlexTemplateSdkClient(Dataflow dataflow) { @@ -47,34 +54,56 @@ public static Builder builder() { return new Builder(); } - public JobInfo launchNewJob(String project, String region, Options options) throws IOException { - LaunchFlexTemplateRequest request = new LaunchFlexTemplateRequest(); - LaunchFlexTemplateParameter parameter = new LaunchFlexTemplateParameter() - .setJobName(options.jobName()) - .setParameters(options.parameters()) - .setContainerSpecGcsPath(options.specPath()); - LaunchFlexTemplateResponse response = client.projects() - .locations() - .flexTemplates() - .launch(project, region, request.setLaunchParameter(parameter)) - .execute(); + @Override + public JobInfo launchNewJob(String project, String region, LaunchOptions options) + throws IOException { + LOG.info("Getting ready to launch {} in {} under {}", options.jobName(), region, project); + LOG.info("Using the spec at {}", options.specPath()); + LOG.info("Using parameters:\n{}", options.parameters()); + + LaunchFlexTemplateParameter parameter = + new LaunchFlexTemplateParameter() + .setJobName(options.jobName()) + .setParameters(options.parameters()) + .setContainerSpecGcsPath(options.specPath()); + LaunchFlexTemplateRequest request = + new LaunchFlexTemplateRequest().setLaunchParameter(parameter); + LOG.info("Sending request:\n{}", request.toPrettyString()); + + LaunchFlexTemplateResponse response = + client.projects().locations().flexTemplates().launch(project, region, request).execute(); + LOG.info("Received response:\n{}", response.toPrettyString()); + Job job = response.getJob(); - return JobInfo.builder() - .setJobId(job.getId()) - .setState(JobState.parse(job.getCurrentState())) - .build(); + // The initial response will not return the state, so need to explicitly get it + JobState state = getJobStatus(project, region, job.getId()); + return JobInfo.builder().setJobId(job.getId()).setState(state).build(); + } + + @Override + public JobState getJobStatus(String project, String region, String jobId) throws IOException { + LOG.info("Getting the status of {} under {}", jobId, project); + + Job job = client.projects().locations().jobs().get(project, region, jobId).execute(); + LOG.info("Received job on get request for {}:\n{}", jobId, job.toPrettyString()); + return handleJobState(job); } - public JobState getJobStatus(String project, String jobId) throws IOException { - Job job = client.projects().jobs().get(project, jobId).execute(); - return JobState.parse(job.getCurrentState()); + @Override + public void cancelJob(String project, String region, String jobId) + throws IOException { + LOG.info("Cancelling {} under {}", jobId, project); + Job job = new Job().setRequestedState(JobState.CANCELLED.toString()); + LOG.info("Sending job to update {}:\n{}", jobId, job.toPrettyString()); + client.projects().locations().jobs().update(project, region, jobId, job).execute(); } - public void cancelJob(String project, String jobId) throws IOException { - Job job = new Job().setCurrentState(JobState.CANCELLED.toString()); - client.projects().jobs().update(project, jobId, job).execute(); + private static JobState handleJobState(Job job) { + String currentState = job.getCurrentState(); + return Strings.isNullOrEmpty(currentState) ? JobState.UNKNOWN : JobState.parse(currentState); } + /** Builder for {@link FlexTemplateSdkClient}. */ public static final class Builder { private Credentials credentials; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java index bc90be24d7..8ab5787140 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java @@ -1 +1,2 @@ -package com.google.cloud.teleport.v2.testing.dataflow; \ No newline at end of file +/** Package for managing Dataflow jobs from integration tests. */ +package com.google.cloud.teleport.v2.testing.dataflow; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java new file mode 100644 index 0000000000..5f78495f17 --- /dev/null +++ b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java @@ -0,0 +1,2 @@ +/** Package for general-purpose testing utilities. */ +package com.google.cloud.teleport.v2.testing; diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java index f6baf3530c..5d77151013 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java @@ -1,3 +1,18 @@ +/* + * Copyright (C) 2022 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.v2.testing; import static com.google.common.truth.Truth.assertThat; @@ -7,11 +22,13 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +/** Unit tests for {@link TestProperties}. */ @RunWith(JUnit4.class) public final class TestPropertiesTest { private static final String ACCESS_TOKEN = "some-token"; private static final String ARTIFACT_BUCKET = "test-bucket"; private static final String PROJECT = "test-project"; + private static final String REGION = "us-east1"; private static final String SPEC_PATH = "gs://test-bucket/some/spec/path"; private final TestProperties properties = new TestProperties(); @@ -21,6 +38,7 @@ public void tearDown() { System.clearProperty(TestProperties.ACCESS_TOKEN_KEY); System.clearProperty(TestProperties.ARTIFACT_BUCKET_KEY); System.clearProperty(TestProperties.PROJECT_KEY); + System.clearProperty(TestProperties.REGION_KEY); System.clearProperty(TestProperties.SPEC_PATH_KEY); } @@ -29,30 +47,31 @@ public void testAllPropertiesSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); assertThat(properties.accessToken()).isEqualTo(ACCESS_TOKEN); assertThat(properties.artifactBucket()).isEqualTo(ARTIFACT_BUCKET); assertThat(properties.project()).isEqualTo(PROJECT); + assertThat(properties.region()).isEqualTo(REGION); assertThat(properties.specPath()).isEqualTo(SPEC_PATH); } - @Test + @Test(expected = IllegalStateException.class) public void testAccessTokenNotSet() { System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - assertThat(properties.accessToken()).isNull(); - assertThat(properties.artifactBucket()).isEqualTo(ARTIFACT_BUCKET); - assertThat(properties.project()).isEqualTo(PROJECT); - assertThat(properties.specPath()).isEqualTo(SPEC_PATH); + properties.accessToken(); } @Test(expected = IllegalStateException.class) public void testArtifactBucketNotSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); properties.artifactBucket(); @@ -62,17 +81,29 @@ public void testArtifactBucketNotSet() { public void testProjectNotSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); + System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); properties.project(); } + @Test + public void testRegionNotSet() { + System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); + System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); + System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); + + assertThat(properties.region()).isEqualTo(TestProperties.DEFAULT_REGION); + } + @Test(expected = IllegalStateException.class) public void testSpecPathNotSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); + System.setProperty(TestProperties.REGION_KEY, REGION); properties.specPath(); } -} \ No newline at end of file +} diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java index 3886f84a22..5abb563387 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java @@ -1,3 +1,18 @@ +/* + * Copyright (C) 2022 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.v2.testing.artifacts; import static com.google.common.truth.Truth.assertThat; @@ -34,6 +49,7 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; +/** Unit tests for {@link ArtifactGcsSdkClient}. */ @RunWith(JUnit4.class) public final class ArtifactGcsSdkClientTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); @@ -43,6 +59,7 @@ public final class ArtifactGcsSdkClientTest { private static final String LOCAL_PATH; private static final byte[] TEST_ARTIFACT_CONTENTS; + static { LOCAL_PATH = Resources.getResource("test-artifact.txt").getPath(); try { @@ -81,8 +98,8 @@ public void testUploadArtifact() throws IOException { @Test(expected = IOException.class) public void testUploadArtifactInvalidLocalPath() throws IOException { when(client.create(any(BlobInfo.class), any())).thenReturn(blob); - new ArtifactGcsSdkClient(client).uploadArtifact( - BUCKET, DIR_PATH, "/some/invalid/path/please/do/not/make/this/file/valid"); + new ArtifactGcsSdkClient(client) + .uploadArtifact(BUCKET, DIR_PATH, "/some/invalid/path/please/do/not/make/this/file/valid"); } @Test @@ -91,8 +108,8 @@ public void testListArtifactsSinglePage() { String name1 = DIR_PATH + "/blob1"; String name2 = DIR_PATH + "/blob2"; String name3 = DIR_PATH + "/blob3"; - ImmutableList page1 = ImmutableList.of( - mock(Blob.class), mock(Blob.class), mock(Blob.class)); + ImmutableList page1 = + ImmutableList.of(mock(Blob.class), mock(Blob.class), mock(Blob.class)); when(page1.get(0).getName()).thenReturn(name1); when(page1.get(1).getName()).thenReturn(name2); when(page1.get(2).getName()).thenReturn(name3); @@ -124,8 +141,7 @@ public void testListArtifactsMultiplePages() { String name1 = DIR_PATH + "/blob1"; String name2 = DIR_PATH + "/blob2"; String name3 = DIR_PATH + "/blob3"; - ImmutableList page1 = ImmutableList.of( - mock(Blob.class), mock(Blob.class)); + ImmutableList page1 = ImmutableList.of(mock(Blob.class), mock(Blob.class)); ImmutableList page2 = ImmutableList.of(mock(Blob.class)); when(page1.get(0).getName()).thenReturn(name1); when(page1.get(1).getName()).thenReturn(name2); @@ -170,8 +186,8 @@ public void testDeleteArtifactsSinglePage() { BlobId id1 = BlobId.of(BUCKET, DIR_PATH + "/blob1"); BlobId id2 = BlobId.of(BUCKET, DIR_PATH + "/blob2"); BlobId id3 = BlobId.of(BUCKET, DIR_PATH + "/blob3"); - ImmutableList page1 = ImmutableList.of( - mock(Blob.class), mock(Blob.class), mock(Blob.class)); + ImmutableList page1 = + ImmutableList.of(mock(Blob.class), mock(Blob.class), mock(Blob.class)); when(page1.get(0).getBlobId()).thenReturn(id1); when(page1.get(1).getBlobId()).thenReturn(id2); when(page1.get(2).getBlobId()).thenReturn(id3); @@ -294,4 +310,4 @@ public Iterable getValues() { return contents; } } -} \ No newline at end of file +} diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java index 3556820f71..0e6823a75e 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java @@ -1,13 +1,31 @@ +/* + * Copyright (C) 2022 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.v2.testing.artifacts; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createGcsClient; import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestDirName; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestDirPath; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestPath; +import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestSuiteDirPath; import static com.google.common.truth.Truth.assertThat; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +/** Artifacts for {@link ArtifactUtils}. */ @RunWith(JUnit4.class) public final class ArtifactUtilsTest { @@ -16,22 +34,31 @@ public final class ArtifactUtilsTest { private static final String TEST_DIR_REGEX = "\\d{8}-[a-fA-F0-9]{8}-([a-fA-F0-9]{4}-){3}[a-fA-F0-9]{12}"; - private static final String BUCKET = "test-bucket"; - private static final String TEST_SUITE_DIR = "artifact-utils-test"; - @Test public void testCreateTestDirName() { assertThat(createTestDirName()).matches(TEST_DIR_REGEX); } @Test - public void testCreateTestDirPath() { - String name = "test-create-test-dir-path"; + public void testCreateTestSuiteDirPath() { + String suiteName = "some-test-class"; + String path = createTestSuiteDirPath(suiteName); + assertThat(path).matches(String.format("%s/%s", suiteName, TEST_DIR_REGEX)); + } + + @Test + public void testCreateTestPath() { + String suiteDirPath = "some/test/suite/dir"; + String testName = "some-test"; - String actual = createTestDirPath(BUCKET, TEST_SUITE_DIR, name); + String path = createTestPath(suiteDirPath, testName); - String expectedPattern = String.format("gs://%s/%s/%s/%s", BUCKET, TEST_SUITE_DIR, name, TEST_DIR_REGEX); + assertThat(path).matches(String.format("%s/%s", suiteDirPath, testName)); + } - assertThat(actual).matches(expectedPattern); + @Test + public void testCreateClientWithNullCredentials() { + createGcsClient(null); + // Just making sure that no exceptions are thrown } -} \ No newline at end of file +} diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java index b3d815e9fb..e8f05b3e6c 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java @@ -1,3 +1,18 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; import static com.google.common.truth.Truth.assertThat; @@ -29,31 +44,36 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; +/** Unit tests for {@link DataflowOperation}. */ @RunWith(JUnit4.class) public final class DataflowOperationTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); - @Mock FlexTemplateClient client; + @Mock private FlexTemplateClient client; private static final String PROJECT = "test-project"; + private static final String REGION = "us-east1"; private static final String JOB_ID = "test-job-id"; private static final Duration CHECK_AFTER = Duration.ofMillis(10); private static final Duration TIMEOUT_AFTER = Duration.ofMillis(100); - private static final Config DEFAULT_CONFIG = Config.builder() - .setProject(PROJECT) - .setJobId(JOB_ID) - .setCheckAfter(CHECK_AFTER) - .setTimeoutAfter(TIMEOUT_AFTER) - .build(); + private static final Config DEFAULT_CONFIG = + Config.builder() + .setProject(PROJECT) + .setRegion(REGION) + .setJobId(JOB_ID) + .setCheckAfter(CHECK_AFTER) + .setTimeoutAfter(TIMEOUT_AFTER) + .build(); - @Captor public ArgumentCaptor projectCaptor; - @Captor public ArgumentCaptor jobIdCaptor; + @Captor private ArgumentCaptor projectCaptor; + @Captor private ArgumentCaptor regionCaptor; + @Captor private ArgumentCaptor jobIdCaptor; @Test public void testWaitUntilDone() throws IOException { // Arrange - when(client.getJobStatus(any(), any())) + when(client.getJobStatus(any(), any(), any())) .thenReturn(JobState.QUEUED) .thenReturn(JobState.RUNNING) .thenReturn(JobState.CANCELLING) @@ -64,19 +84,21 @@ public void testWaitUntilDone() throws IOException { // Assert verify(client, times(4)) - .getJobStatus(projectCaptor.capture(), jobIdCaptor.capture()); + .getJobStatus(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); Set allProjects = new HashSet<>(projectCaptor.getAllValues()); + Set allRegions = new HashSet<>(regionCaptor.getAllValues()); Set allJobIds = new HashSet<>(jobIdCaptor.getAllValues()); assertThat(allProjects).containsExactly(PROJECT); + assertThat(allRegions).containsExactly(REGION); assertThat(allJobIds).containsExactly(JOB_ID); assertThat(result).isEqualTo(Result.JOB_FINISHED); } @Test public void testWaitUntilDoneTimeout() throws IOException { - when(client.getJobStatus(any(), any())).thenReturn(JobState.RUNNING); + when(client.getJobStatus(any(), any(), any())).thenReturn(JobState.RUNNING); Result result = DataflowOperation.waitUntilDone(client, DEFAULT_CONFIG); assertThat(result).isEqualTo(Result.TIMEOUT); } @@ -86,22 +108,24 @@ public void testWaitForCondition() throws IOException { AtomicInteger callCount = new AtomicInteger(); int totalCalls = 3; Supplier checker = () -> callCount.incrementAndGet() >= totalCalls; - when(client.getJobStatus(any(), any())) + when(client.getJobStatus(any(), any(), any())) .thenReturn(JobState.RUNNING) .thenThrow(new IOException()) .thenReturn(JobState.RUNNING); Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, checker); - verify(client, atMost(totalCalls)).getJobStatus(projectCaptor.capture(), jobIdCaptor.capture()); + verify(client, atMost(totalCalls)) + .getJobStatus(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); assertThat(result).isEqualTo(Result.CONDITION_MET); } @Test public void testWaitForConditionJobFinished() throws IOException { - when(client.getJobStatus(any(), any())) + when(client.getJobStatus(any(), any(), any())) .thenReturn(JobState.RUNNING) .thenReturn(JobState.CANCELLED); @@ -112,7 +136,7 @@ public void testWaitForConditionJobFinished() throws IOException { @Test public void testWaitForConditionTimeout() throws IOException { - when(client.getJobStatus(any(), any())).thenReturn(JobState.RUNNING); + when(client.getJobStatus(any(), any(), any())).thenReturn(JobState.RUNNING); Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); @@ -126,52 +150,59 @@ public void testFinishAfterCondition() throws IOException { int totalCalls = 3; Supplier checker = () -> callCount.incrementAndGet() >= totalCalls; - when(client.getJobStatus(any(), any())) + when(client.getJobStatus(any(), any(), any())) .thenReturn(JobState.RUNNING) .thenThrow(new IOException()) .thenReturn(JobState.RUNNING) .thenReturn(JobState.CANCELLING) .thenReturn(JobState.CANCELLED); - doAnswer(invocation -> null).when(client).cancelJob(any(), any()); + doAnswer(invocation -> null).when(client).cancelJob(any(), any(), any()); // Act Result result = DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, checker); // Assert verify(client, atLeast(totalCalls)) - .getJobStatus(projectCaptor.capture(), jobIdCaptor.capture()); - verify(client).cancelJob(projectCaptor.capture(), jobIdCaptor.capture()); + .getJobStatus(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); + verify(client) + .cancelJob( + projectCaptor.capture(), + regionCaptor.capture(), + jobIdCaptor.capture()); Set allProjects = new HashSet<>(projectCaptor.getAllValues()); + Set allRegions = new HashSet<>(regionCaptor.getAllValues()); Set allJobIds = new HashSet<>(jobIdCaptor.getAllValues()); assertThat(allProjects).containsExactly(PROJECT); + assertThat(allRegions).containsExactly(REGION); assertThat(allJobIds).containsExactly(JOB_ID); assertThat(result).isEqualTo(Result.CONDITION_MET); } @Test public void testFinishAfterConditionJobStopped() throws IOException { - when(client.getJobStatus(any(), any())) + when(client.getJobStatus(any(), any(), any())) .thenReturn(JobState.RUNNING) .thenReturn(JobState.CANCELLED); - doAnswer(invocation -> null).when(client).cancelJob(projectCaptor.capture(), jobIdCaptor.capture()); + doAnswer(invocation -> null).when(client).cancelJob(any(), any(), any()); - Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + Result result = + DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, () -> false); - verify(client, never()).cancelJob(any(), any()); + verify(client, never()).cancelJob(any(), any(), any()); assertThat(result).isEqualTo(Result.JOB_FINISHED); } @Test public void testFinishAfterConditionTimeout() throws IOException { - when(client.getJobStatus(any(), any())) - .thenReturn(JobState.RUNNING); - doAnswer(invocation -> null).when(client).cancelJob(projectCaptor.capture(), jobIdCaptor.capture()); + when(client.getJobStatus(any(), any(), any())).thenReturn(JobState.RUNNING); + doAnswer(invocation -> null).when(client).cancelJob(any(), any(), any()); - Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + Result result = + DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, () -> false); - verify(client, never()).cancelJob(any(), any()); + verify(client).cancelJob(any(), any(), any()); assertThat(result).isEqualTo(Result.TIMEOUT); } -} \ No newline at end of file +} diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java index 50ed81aaac..0384086f26 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java @@ -1,3 +1,18 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; import static com.google.cloud.teleport.v2.testing.dataflow.DataflowUtils.createJobName; @@ -7,6 +22,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +/** Unit tests for {@link DataflowUtils}. */ @RunWith(JUnit4.class) public class DataflowUtilsTest { @@ -15,4 +31,4 @@ public void testCreateJobName() { String name = "create-job-name"; assertThat(createJobName(name)).matches(name + "-\\d{14}"); } -} \ No newline at end of file +} diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java index 11d0bd725d..ea2fafd312 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java +++ b/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java @@ -1,16 +1,32 @@ +/* + * Copyright (C) 2022 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.v2.testing.dataflow; import static com.google.common.truth.Truth.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.Dataflow.Projects; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Update; +import com.google.api.services.dataflow.Dataflow.Projects.Locations; import com.google.api.services.dataflow.Dataflow.Projects.Locations.FlexTemplates; import com.google.api.services.dataflow.Dataflow.Projects.Locations.FlexTemplates.Launch; +import com.google.api.services.dataflow.Dataflow.Projects.Locations.Jobs.Get; +import com.google.api.services.dataflow.Dataflow.Projects.Locations.Jobs.Update; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.LaunchFlexTemplateParameter; import com.google.api.services.dataflow.model.LaunchFlexTemplateRequest; @@ -18,7 +34,7 @@ import com.google.auth.Credentials; import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobInfo; import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.Options; +import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.LaunchOptions; import com.google.common.collect.ImmutableMap; import java.io.IOException; import org.junit.Rule; @@ -32,11 +48,13 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; +/** Unit test for {@link FlexTemplateSdkClient}. */ @RunWith(JUnit4.class) -public final class FlexTemplateSdkClientTest{ +public final class FlexTemplateSdkClientTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); - @Mock(answer = Answers.RETURNS_DEEP_STUBS) private Dataflow client; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Dataflow client; private static final String PROJECT = "test-project"; private static final String REGION = "us-east1"; @@ -60,98 +78,113 @@ public void testCreateWithCredentials() { // Lack of exception is all we really can test } - @Test - public void testCreateWithNullCredentials() { - FlexTemplateSdkClient.builder().setCredentials(null).build(); - // Lack of exception is all we really can test - } - @Test public void testLaunchNewJob() throws IOException { // Arrange Launch launch = mock(Launch.class); - Job job = new Job().setId(JOB_ID).setCurrentState(JobState.QUEUED.toString()); - LaunchFlexTemplateResponse response = new LaunchFlexTemplateResponse().setJob(job); + Get get = mock(Get.class); + Job launchJob = new Job().setId(JOB_ID); + Job getJob = new Job().setId(JOB_ID).setCurrentState(JobState.QUEUED.toString()); + LaunchFlexTemplateResponse response = new LaunchFlexTemplateResponse().setJob(launchJob); - Options options = new Options(JOB_NAME, SPEC_PATH) - .setIsStreaming(true) - .addParameter(PARAM_KEY, PARAM_VALUE); + LaunchOptions options = + LaunchOptions.builder(JOB_NAME, SPEC_PATH).addParameter(PARAM_KEY, PARAM_VALUE).build(); - when(getFlexTemplates(client).launch(projectCaptor.capture(), regionCaptor.capture(), requestCaptor.capture())).thenReturn(launch); + when(getFlexTemplates(client).launch(any(), any(), any())).thenReturn(launch); + when(getLocationJobs(client).get(any(), any(), any())).thenReturn(get); when(launch.execute()).thenReturn(response); + when(get.execute()).thenReturn(getJob); // Act - JobInfo actual = FlexTemplateSdkClient.withDataflowClient(client).launchNewJob(PROJECT, REGION, options); + JobInfo actual = + FlexTemplateSdkClient.withDataflowClient(client).launchNewJob(PROJECT, REGION, options); // Assert - JobInfo expected = JobInfo.builder() - .setJobId(JOB_ID) - .setState(JobState.QUEUED) - .build(); - - LaunchFlexTemplateRequest expectedRequest = new LaunchFlexTemplateRequest() - .setLaunchParameter(new LaunchFlexTemplateParameter() - .setJobName(JOB_NAME) - .setContainerSpecGcsPath(SPEC_PATH) - .setParameters(ImmutableMap.of(PARAM_KEY, PARAM_VALUE, "isStreaming", "true"))); - - assertThat(actual).isEqualTo(expected); + LaunchFlexTemplateRequest expectedRequest = + new LaunchFlexTemplateRequest() + .setLaunchParameter( + new LaunchFlexTemplateParameter() + .setJobName(JOB_NAME) + .setContainerSpecGcsPath(SPEC_PATH) + .setParameters(ImmutableMap.of(PARAM_KEY, PARAM_VALUE))); + verify(getFlexTemplates(client)) + .launch(projectCaptor.capture(), regionCaptor.capture(), requestCaptor.capture()); assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); assertThat(regionCaptor.getValue()).isEqualTo(REGION); assertThat(requestCaptor.getValue()).isEqualTo(expectedRequest); + + verify(getLocationJobs(client)) + .get(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); + assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + + JobInfo expected = JobInfo.builder().setJobId(JOB_ID).setState(JobState.QUEUED).build(); + assertThat(actual).isEqualTo(expected); } @Test(expected = IOException.class) public void testLaunchNewJobThrowsException() throws IOException { when(getFlexTemplates(client).launch(any(), any(), any())).thenThrow(new IOException()); - FlexTemplateSdkClient.withDataflowClient(client).launchNewJob(PROJECT, REGION, new Options(JOB_NAME, SPEC_PATH)); + FlexTemplateSdkClient.withDataflowClient(client) + .launchNewJob(PROJECT, REGION, LaunchOptions.builder(JOB_NAME, SPEC_PATH).build()); } @Test public void testGetJobStatus() throws IOException { Get get = mock(Get.class); Job job = new Job().setCurrentState(JobState.RUNNING.toString()); - when(getProjectJobs(client).get(projectCaptor.capture(), jobIdCaptor.capture())).thenReturn(get); + when(getLocationJobs(client).get(any(), any(), any())).thenReturn(get); when(get.execute()).thenReturn(job); - JobState actual = FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, JOB_ID); + JobState actual = + FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, REGION, JOB_ID); - assertThat(actual).isEqualTo(JobState.RUNNING); + verify(getLocationJobs(client)) + .get(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + assertThat(actual).isEqualTo(JobState.RUNNING); } @Test(expected = IOException.class) public void testGetJobThrowsException() throws IOException { - when(getProjectJobs(client).get(any(), any())).thenThrow(new IOException()); - FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, JOB_ID); + when(getLocationJobs(client).get(any(), any(), any())).thenThrow(new IOException()); + FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, REGION, JOB_ID); } @Test public void testCancelJob() throws IOException { Update update = mock(Update.class); - when(getProjectJobs(client).update(projectCaptor.capture(), jobIdCaptor.capture(), - jobCaptor.capture())).thenReturn(update); + when(getLocationJobs(client).update(any(), any(), any(), any())).thenReturn(update); when(update.execute()).thenReturn(new Job()); - FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, JOB_ID); + FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, REGION, JOB_ID); + verify(getLocationJobs(client)) + .update( + projectCaptor.capture(), + regionCaptor.capture(), + jobIdCaptor.capture(), + jobCaptor.capture()); assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); - assertThat(jobCaptor.getValue().getCurrentState()).isEqualTo(JobState.CANCELLED.toString()); + assertThat(jobCaptor.getValue().getRequestedState()).isEqualTo(JobState.CANCELLED.toString()); } @Test(expected = IOException.class) public void testCancelJobThrowsException() throws IOException { - when(getProjectJobs(client).update(any(), any(), any())).thenThrow(new IOException()); - FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, JOB_ID); + when(getLocationJobs(client).update(any(), any(), any(), any())).thenThrow(new IOException()); + FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, REGION, JOB_ID); } - private static Projects.Jobs getProjectJobs(Dataflow client) { - return client.projects().jobs(); + private static Locations.Jobs getLocationJobs(Dataflow client) { + return client.projects().locations().jobs(); } private static FlexTemplates getFlexTemplates(Dataflow client) { return client.projects().locations().flexTemplates(); } -} \ No newline at end of file +} From ad7b667c497b583f64ef4ce2bf9813dabf5a3d12 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 27 Jan 2022 10:49:23 -0500 Subject: [PATCH 070/145] Make it easier to reuse templates client for Classic templates --- cicd/internal/repo/modules.go | 5 + cicd/internal/repo/modules_test.go | 9 +- it/JAVA_LICENSE_HEADER | 15 + it/checkstyle/checkstyle.xml | 319 ++++++++++++++++++ it/checkstyle/suppressions.xml | 42 +++ it/pom.xml | 212 ++++++++++++ .../cloud/teleport/it}/TestProperties.java | 4 +- .../it}/artifacts/ArtifactClient.java | 2 +- .../it}/artifacts/ArtifactGcsSdkClient.java | 6 +- .../teleport/it}/artifacts/ArtifactUtils.java | 2 +- .../teleport/it/artifacts/package-info.java | 18 + .../AbstractDataflowTemplateClient.java | 63 ++++ .../it/dataflow/DataflowOperator.java | 59 ++-- .../it/dataflow/DataflowTemplateClient.java | 17 +- .../teleport/it}/dataflow/DataflowUtils.java | 2 +- .../it/dataflow/FlexTemplateClient.java | 62 ++-- .../teleport/it/dataflow/package-info.java | 18 + .../cloud/teleport/it/logging/LogStrings.java | 61 ++++ .../teleport/it/logging/package-info.java | 18 + .../cloud/teleport/it/package-info.java | 18 + .../src/main/resources/test-artifact.txt | 0 .../teleport/it}/TestPropertiesTest.java | 19 +- .../artifacts/ArtifactGcsSdkClientTest.java | 17 +- .../it}/artifacts/ArtifactUtilsTest.java | 15 +- .../AbstractDataflowTemplateClientTest.java | 131 +++++++ .../it/dataflow/DataflowOperatorTest.java | 35 +- .../it}/dataflow/DataflowUtilsTest.java | 4 +- .../it/dataflow/FlexTemplateClientTest.java | 79 +---- v2/pom.xml | 7 +- v2/streaming-data-generator/pom.xml | 6 - .../templates/StreamingDataGeneratorIT.java | 56 +-- v2/testing/pom.xml | 33 -- .../v2/testing/artifacts/package-info.java | 2 - .../v2/testing/dataflow/package-info.java | 2 - .../teleport/v2/testing/package-info.java | 2 - 35 files changed, 1086 insertions(+), 274 deletions(-) create mode 100644 it/JAVA_LICENSE_HEADER create mode 100644 it/checkstyle/checkstyle.xml create mode 100644 it/checkstyle/suppressions.xml create mode 100644 it/pom.xml rename {v2/testing/src/main/java/com/google/cloud/teleport/v2/testing => it/src/main/java/com/google/cloud/teleport/it}/TestProperties.java (95%) rename {v2/testing/src/main/java/com/google/cloud/teleport/v2/testing => it/src/main/java/com/google/cloud/teleport/it}/artifacts/ArtifactClient.java (97%) rename {v2/testing/src/main/java/com/google/cloud/teleport/v2/testing => it/src/main/java/com/google/cloud/teleport/it}/artifacts/ArtifactGcsSdkClient.java (93%) rename {v2/testing/src/main/java/com/google/cloud/teleport/v2/testing => it/src/main/java/com/google/cloud/teleport/it}/artifacts/ArtifactUtils.java (97%) create mode 100644 it/src/main/java/com/google/cloud/teleport/it/artifacts/package-info.java create mode 100644 it/src/main/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClient.java rename v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java => it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowOperator.java (73%) rename v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java => it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java (88%) rename {v2/testing/src/main/java/com/google/cloud/teleport/v2/testing => it/src/main/java/com/google/cloud/teleport/it}/dataflow/DataflowUtils.java (96%) rename v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java => it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java (59%) create mode 100644 it/src/main/java/com/google/cloud/teleport/it/dataflow/package-info.java create mode 100644 it/src/main/java/com/google/cloud/teleport/it/logging/LogStrings.java create mode 100644 it/src/main/java/com/google/cloud/teleport/it/logging/package-info.java create mode 100644 it/src/main/java/com/google/cloud/teleport/it/package-info.java rename {v2/testing => it}/src/main/resources/test-artifact.txt (100%) rename {v2/testing/src/test/java/com/google/cloud/teleport/v2/testing => it/src/test/java/com/google/cloud/teleport/it}/TestPropertiesTest.java (90%) rename {v2/testing/src/test/java/com/google/cloud/teleport/v2/testing => it/src/test/java/com/google/cloud/teleport/it}/artifacts/ArtifactGcsSdkClientTest.java (96%) rename {v2/testing/src/test/java/com/google/cloud/teleport/v2/testing => it/src/test/java/com/google/cloud/teleport/it}/artifacts/ArtifactUtilsTest.java (72%) create mode 100644 it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java rename v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java => it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java (84%) rename {v2/testing/src/test/java/com/google/cloud/teleport/v2/testing => it/src/test/java/com/google/cloud/teleport/it}/dataflow/DataflowUtilsTest.java (87%) rename v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java => it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java (63%) delete mode 100644 v2/testing/pom.xml delete mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java delete mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java delete mode 100644 v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index d81dd0ca6e..c3bc477fd6 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -27,6 +27,7 @@ const ( // Roots in relation to the root directory of the repository. ClassicRoot = "." FlexRoot = "v2" + ItRoot = "it" ) // Returns all of the known roots modules. @@ -46,6 +47,7 @@ func GetAllRoots() []string { func GetModuleMapping() map[string][]string { m := make(map[string][]string) m[ClassicRoot] = make([]string, 0) + m[ItRoot] = make([]string, 0) flexPoms, err := GetAllPomFiles(FlexRoot) if err != nil { @@ -99,11 +101,14 @@ func GetModulesForPaths(paths []string) map[string][]string { m := make(map[string][]string) flex := make([]string, 0) + it := fmt.Sprintf("it%s", string(os.PathSeparator)) v2 := fmt.Sprintf("v2%s", string(os.PathSeparator)) for _, path := range paths { if strings.HasPrefix(path, v2) { flex = append(flex, strings.TrimPrefix(path, v2)) + } else if strings.HasPrefix(path, it) { + m[ItRoot] = make([]string, 0) } else { // TODO(zhoufek): Make this more granular, especially separating .github and cicd code // into separate "modules" diff --git a/cicd/internal/repo/modules_test.go b/cicd/internal/repo/modules_test.go index 2545c5252a..b4185dd9b6 100644 --- a/cicd/internal/repo/modules_test.go +++ b/cicd/internal/repo/modules_test.go @@ -34,6 +34,9 @@ func TestModuleMappingHasAllRoots(t *testing.T) { if _, ok := m[ClassicRoot]; !ok { t.Error("Missing Classic root") } + if _, ok := m[ItRoot]; !ok { + t.Error("Missing integration test root") + } if _, ok := m[FlexRoot]; !ok { t.Error("Missing Flex root") } @@ -55,7 +58,8 @@ func TestGetModulesForPaths(t *testing.T) { }, expected: map[string][]string{ ClassicRoot: []string{}, - FlexRoot: []string{"pubsub-binary-to-bigquery", ""}, + ItRoot: []string{}, + FlexRoot: []string{"pubsub-binary-to-bigquery"}, }, }, { @@ -74,7 +78,8 @@ func TestGetModulesForPaths(t *testing.T) { input: []string{"something", "it/something", "v2/something"}, expected: map[string][]string{ ClassicRoot: make([]string, 0), - FlexRoot: []string{""}, + ItRoot: make([]string, 0), + FlexRoot: make([]string, 0), }, }, { diff --git a/it/JAVA_LICENSE_HEADER b/it/JAVA_LICENSE_HEADER new file mode 100644 index 0000000000..91cf64415d --- /dev/null +++ b/it/JAVA_LICENSE_HEADER @@ -0,0 +1,15 @@ +/* + * Copyright (C) $YEAR 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. + */ \ No newline at end of file diff --git a/it/checkstyle/checkstyle.xml b/it/checkstyle/checkstyle.xml new file mode 100644 index 0000000000..599e4569ad --- /dev/null +++ b/it/checkstyle/checkstyle.xml @@ -0,0 +1,319 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/it/checkstyle/suppressions.xml b/it/checkstyle/suppressions.xml new file mode 100644 index 0000000000..5e962ac521 --- /dev/null +++ b/it/checkstyle/suppressions.xml @@ -0,0 +1,42 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/it/pom.xml b/it/pom.xml new file mode 100644 index 0000000000..771d20d469 --- /dev/null +++ b/it/pom.xml @@ -0,0 +1,212 @@ + + + + com.google.cloud.teleport + it + 1.0-SNAPSHOT + 4.0.0 + + + 1.8.2 + 2.34.0 + 24.2.0 + 1.8 + 4.13.2 + 31.0.1-jre + 3.0.0 + 3.6.2 + 3.7.7 + 8.7 + 1.5 + 1.7.25 + 2.12.1 + 2.21.0 + 1.0.1 + + + + + + com.google.cloud + libraries-bom + ${cloud-bom.version} + pom + import + + + + + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${beam.version} + + + + + com.google.cloud + google-cloud-storage + + + + com.google.auto.value + auto-value + ${autovalue.version} + provided + + + com.google.auto.value + auto-value-annotations + ${autovalue.version} + + + com.google.guava + guava + ${guava.version} + + + com.google.re2j + re2j + ${re2j.version} + + + org.slf4j + slf4j-simple + ${slf4j.version} + + + + + com.google.truth + truth + ${truth.version} + test + + + junit + junit + ${junit.version} + test + + + org.mockito + mockito-core + ${mockito.version} + test + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + ${java.version} + ${java.version} + + true + -parameters + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire.version} + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + ${maven-checkstyle-plugin.version} + + + com.puppycrawl.tools + checkstyle + ${puppycrawl.version} + + + + checkstyle/checkstyle.xml + checkstyle/suppressions.xml + true + true + false + true + + + + + test-compile + + check + + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless-maven-plugin.version} + + + + + + + *.md + + + + + + + + + 1.8 + + + + JAVA_LICENSE_HEADER + + + + + + + + check + + + + + + + + \ No newline at end of file diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java b/it/src/main/java/com/google/cloud/teleport/it/TestProperties.java similarity index 95% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java rename to it/src/main/java/com/google/cloud/teleport/it/TestProperties.java index 6d6a10e57e..c0b0451d60 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/TestProperties.java +++ b/it/src/main/java/com/google/cloud/teleport/it/TestProperties.java @@ -13,9 +13,9 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing; +package com.google.cloud.teleport.it; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Preconditions.checkState; import com.google.auth.Credentials; import com.google.auth.oauth2.AccessToken; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java similarity index 97% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java rename to it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java index 672e7685e1..66224260f7 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.artifacts; +package com.google.cloud.teleport.it.artifacts; import com.google.cloud.storage.Blob; import com.google.re2j.Pattern; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java similarity index 93% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java rename to it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java index cdd0f028b1..62096fed0d 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java @@ -13,9 +13,9 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.artifacts; +package com.google.cloud.teleport.it.artifacts; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableList.toImmutableList; import com.google.api.gax.paging.Page; import com.google.cloud.storage.Blob; @@ -23,6 +23,7 @@ import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; import com.google.cloud.storage.Storage.BlobListOption; +import com.google.common.collect.ImmutableList; import com.google.re2j.Pattern; import java.io.IOException; import java.nio.file.Files; @@ -31,7 +32,6 @@ import java.util.List; import java.util.function.Consumer; import java.util.stream.StreamSupport; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java similarity index 97% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java rename to it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java index 4af07e3813..074fb75e12 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtils.java +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.artifacts; +package com.google.cloud.teleport.it.artifacts; import com.google.auth.Credentials; import com.google.cloud.storage.Storage; diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/package-info.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/package-info.java new file mode 100644 index 0000000000..38ec7bc7b1 --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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 for working with test artifacts. */ +package com.google.cloud.teleport.it.artifacts; diff --git a/it/src/main/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClient.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClient.java new file mode 100644 index 0000000000..f48c3550ab --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClient.java @@ -0,0 +1,63 @@ +/* + * Copyright (C) 2022 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.it.dataflow; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.Job; +import com.google.cloud.teleport.it.logging.LogStrings; +import com.google.common.base.Strings; +import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Abstract class covering the common operations between Classic and Flex templates. + * + *

Generally, the methods here are the ones that focus more on the Dataflow jobs rather than + * launching a specific type of template. + */ +abstract class AbstractDataflowTemplateClient implements DataflowTemplateClient { + private static final Logger LOG = LoggerFactory.getLogger(AbstractDataflowTemplateClient.class); + + protected final Dataflow client; + + AbstractDataflowTemplateClient(Dataflow client) { + this.client = client; + } + + @Override + public JobState getJobStatus(String project, String region, String jobId) throws IOException { + LOG.info("Getting the status of {} under {}", jobId, project); + + Job job = client.projects().locations().jobs().get(project, region, jobId).execute(); + LOG.info("Received job on get request for {}:\n{}", jobId, LogStrings.formatForLogging(job)); + return handleJobState(job); + } + + @Override + public void cancelJob(String project, String region, String jobId) throws IOException { + LOG.info("Cancelling {} under {}", jobId, project); + Job job = new Job().setRequestedState(JobState.CANCELLED.toString()); + LOG.info("Sending job to update {}:\n{}", jobId, LogStrings.formatForLogging(job)); + client.projects().locations().jobs().update(project, region, jobId, job).execute(); + } + + /** Parses the job state if available or returns {@link JobState#UNKNOWN} if not given. */ + protected static JobState handleJobState(Job job) { + String currentState = job.getCurrentState(); + return Strings.isNullOrEmpty(currentState) ? JobState.UNKNOWN : JobState.parse(currentState); + } +} diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowOperator.java similarity index 73% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java rename to it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowOperator.java index 76b7aa44f5..88d2ff9ed6 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperation.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowOperator.java @@ -13,24 +13,23 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; +import com.google.common.base.Strings; import java.io.IOException; import java.time.Duration; import java.time.Instant; import java.util.function.Supplier; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Utilities for managing Dataflow jobs. */ -public final class DataflowOperation { - private static final Logger LOG = LoggerFactory.getLogger(DataflowOperation.class); +public final class DataflowOperator { + private static final Logger LOG = LoggerFactory.getLogger(DataflowOperator.class); /** The result of running an operation. */ public enum Result { @@ -39,14 +38,11 @@ public enum Result { TIMEOUT } - public static final ImmutableSet DONE_STATES = - ImmutableSet.of( - JobState.CANCELLED, JobState.DONE, JobState.DRAINED, JobState.FAILED, JobState.STOPPED); + private final DataflowTemplateClient client; - public static final ImmutableSet FINISHING_STATES = - ImmutableSet.of(JobState.DRAINING, JobState.CANCELLING); - - private DataflowOperation() {} + public DataflowOperator(DataflowTemplateClient client) { + this.client = client; + } /** * Waits until the given job is done, timing out it if runs for too long. @@ -54,33 +50,28 @@ private DataflowOperation() {} *

If the job is a batch job, it should complete eventually. If it is a streaming job, this * will time out unless the job is explicitly cancelled or drained. * - * @param client the {@link FlexTemplateClient} to use for performing operations * @param config the configuration for performing the operation * @return the result, which will be either {@link Result#JOB_FINISHED} or {@link Result#TIMEOUT} */ - public static Result waitUntilDone(FlexTemplateClient client, Config config) { + public Result waitUntilDone(Config config) { return finishOrTimeout( - config, - () -> false, - () -> jobIsDone(client, config.project(), config.region(), config.jobId())); + config, () -> false, () -> jobIsDone(config.project(), config.region(), config.jobId())); } /** * Waits until a given condition is met OR when the job enters a state that indicates that it is * done or ready to be done. * - * @param client the {@link FlexTemplateClient} to use for performing operations * @param config the configuration for performing operations * @param conditionCheck a {@link Supplier} that will be called periodically to check if the * condition is met * @return the result, which could be any value in {@link Result} */ - public static Result waitForCondition( - FlexTemplateClient client, Config config, Supplier conditionCheck) { + public Result waitForCondition(Config config, Supplier conditionCheck) { return finishOrTimeout( config, conditionCheck, - () -> jobIsDoneOrFinishing(client, config.project(), config.region(), config.jobId())); + () -> jobIsDoneOrFinishing(config.project(), config.region(), config.jobId())); } /** @@ -90,20 +81,18 @@ public static Result waitForCondition( *

If the condition was met before the job entered a done or finishing state, then this will * cancel the job and wait for the job to enter a done state. * - * @param client the {@link FlexTemplateClient} to use for performing operations * @param config the configuration for performing operations * @param conditionCheck a {@link Supplier} that will be called periodically to check if the * condition is met * @return the result of waiting for the condition, not of waiting for the job to be done * @throws IOException if there is an issue cancelling the job */ - public static Result waitForConditionAndFinish( - FlexTemplateClient client, Config config, Supplier conditionCheck) + public Result waitForConditionAndFinish(Config config, Supplier conditionCheck) throws IOException { - Result conditionStatus = waitForCondition(client, config, conditionCheck); + Result conditionStatus = waitForCondition(config, conditionCheck); if (conditionStatus != Result.JOB_FINISHED) { client.cancelJob(config.project(), config.region(), config.jobId()); - waitUntilDone(client, config); + waitUntilDone(config); } return conditionStatus; } @@ -127,10 +116,12 @@ private static Result finishOrTimeout( LOG.info("Checking if condition is met."); if (conditionCheck.get()) { + LOG.info("Condition met!"); return Result.CONDITION_MET; } LOG.info("Condition not met. Checking if job is finished."); if (stopChecking.get()) { + LOG.info("Detected that we should stop checking."); return Result.JOB_FINISHED; } LOG.info( @@ -141,24 +132,22 @@ private static Result finishOrTimeout( return Result.TIMEOUT; } - private static boolean jobIsDone( - FlexTemplateClient client, String project, String region, String jobId) { + private boolean jobIsDone(String project, String region, String jobId) { try { JobState state = client.getJobStatus(project, region, jobId); LOG.info("Job is in state {}", state); - return DONE_STATES.contains(state); + return JobState.DONE_STATES.contains(state); } catch (IOException e) { LOG.error("Failed to get current job state. Assuming not done.", e); return false; } } - private static boolean jobIsDoneOrFinishing( - FlexTemplateClient client, String project, String region, String jobId) { + private boolean jobIsDoneOrFinishing(String project, String region, String jobId) { try { JobState state = client.getJobStatus(project, region, jobId); LOG.info("Job is in state {}", state); - return DONE_STATES.contains(state) || FINISHING_STATES.contains(state); + return JobState.DONE_STATES.contains(state) || JobState.FINISHING_STATES.contains(state); } catch (IOException e) { LOG.error("Failed to get current job state. Assuming not done.", e); return false; @@ -184,7 +173,7 @@ public abstract static class Config { // TODO(zhoufek): Also let users set the maximum number of exceptions. public static Builder builder() { - return new AutoValue_DataflowOperation_Config.Builder() + return new AutoValue_DataflowOperator_Config.Builder() .setCheckAfter(Duration.ofSeconds(30)) .setTimeoutAfter(Duration.ofMinutes(15)); } diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java similarity index 88% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java rename to it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java index 88fff78916..f228b61dd8 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableMap; @@ -24,7 +24,7 @@ import javax.annotation.Nullable; /** Client for working with Flex templates. */ -public interface FlexTemplateClient { +public interface DataflowTemplateClient { /** Enum representing known Dataflow job states. */ enum JobState { UNKNOWN("JOB_STATE_UNKNOWN"), @@ -43,9 +43,18 @@ enum JobState { private static final String DATAFLOW_PREFIX = "JOB_STATE_"; + /** States that indicate the job is running or getting ready to run. */ public static final ImmutableSet RUNNING_STATES = ImmutableSet.of(RUNNING, UPDATED, PENDING, QUEUED); + /** States that indicate that the job is done. */ + public static final ImmutableSet DONE_STATES = + ImmutableSet.of(CANCELLED, DONE, DRAINED, FAILED, STOPPED); + + /** States that indicate that the job is in the process of finishing. */ + public static final ImmutableSet FINISHING_STATES = + ImmutableSet.of(DRAINING, CANCELLING); + private final String text; JobState(String text) { @@ -139,7 +148,7 @@ abstract class JobInfo { public abstract JobState state(); public static Builder builder() { - return new AutoValue_FlexTemplateClient_JobInfo.Builder(); + return new AutoValue_DataflowTemplateClient_JobInfo.Builder(); } /** Builder for {@link JobInfo}. */ @@ -162,7 +171,7 @@ public abstract static class Builder { * @return info about the request to launch a new job * @throws IOException if there is an issue sending the request */ - JobInfo launchNewJob(String project, String region, LaunchOptions options) throws IOException; + JobInfo launchTemplate(String project, String region, LaunchOptions options) throws IOException; /** * Gets the current status of a job. diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java similarity index 96% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java rename to it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java index 152629ec63..09d918d70e 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtils.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; import java.time.Instant; import java.time.ZoneId; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java similarity index 59% rename from v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java rename to it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java index 30df83118b..596615a5dd 100644 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java @@ -13,7 +13,9 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; + +import static com.google.cloud.teleport.it.logging.LogStrings.formatForLogging; import com.google.api.client.googleapis.util.Utils; import com.google.api.services.dataflow.Dataflow; @@ -23,31 +25,28 @@ import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse; import com.google.auth.Credentials; import com.google.auth.http.HttpCredentialsAdapter; -import com.google.common.base.Strings; import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Client for interacting with Dataflow Flex Templates using the Dataflow SDK. */ -public final class FlexTemplateSdkClient implements FlexTemplateClient { - private static final Logger LOG = LoggerFactory.getLogger(FlexTemplateSdkClient.class); - - private final Dataflow client; +public final class FlexTemplateClient extends AbstractDataflowTemplateClient { + private static final Logger LOG = LoggerFactory.getLogger(FlexTemplateClient.class); - private FlexTemplateSdkClient(Builder builder) { - this.client = + private FlexTemplateClient(Builder builder) { + super( new Dataflow( Utils.getDefaultTransport(), Utils.getDefaultJsonFactory(), - new HttpCredentialsAdapter(builder.getCredentials())); + new HttpCredentialsAdapter(builder.getCredentials()))); } - private FlexTemplateSdkClient(Dataflow dataflow) { - this.client = dataflow; + private FlexTemplateClient(Dataflow dataflow) { + super(dataflow); } - public static FlexTemplateSdkClient withDataflowClient(Dataflow dataflow) { - return new FlexTemplateSdkClient(dataflow); + public static FlexTemplateClient withDataflowClient(Dataflow dataflow) { + return new FlexTemplateClient(dataflow); } public static Builder builder() { @@ -55,11 +54,11 @@ public static Builder builder() { } @Override - public JobInfo launchNewJob(String project, String region, LaunchOptions options) + public JobInfo launchTemplate(String project, String region, LaunchOptions options) throws IOException { LOG.info("Getting ready to launch {} in {} under {}", options.jobName(), region, project); LOG.info("Using the spec at {}", options.specPath()); - LOG.info("Using parameters:\n{}", options.parameters()); + LOG.info("Using parameters:\n{}", formatForLogging(options.parameters())); LaunchFlexTemplateParameter parameter = new LaunchFlexTemplateParameter() @@ -68,11 +67,11 @@ public JobInfo launchNewJob(String project, String region, LaunchOptions options .setContainerSpecGcsPath(options.specPath()); LaunchFlexTemplateRequest request = new LaunchFlexTemplateRequest().setLaunchParameter(parameter); - LOG.info("Sending request:\n{}", request.toPrettyString()); + LOG.info("Sending request:\n{}", formatForLogging(request)); LaunchFlexTemplateResponse response = client.projects().locations().flexTemplates().launch(project, region, request).execute(); - LOG.info("Received response:\n{}", response.toPrettyString()); + LOG.info("Received response:\n{}", formatForLogging(response)); Job job = response.getJob(); // The initial response will not return the state, so need to explicitly get it @@ -80,30 +79,7 @@ public JobInfo launchNewJob(String project, String region, LaunchOptions options return JobInfo.builder().setJobId(job.getId()).setState(state).build(); } - @Override - public JobState getJobStatus(String project, String region, String jobId) throws IOException { - LOG.info("Getting the status of {} under {}", jobId, project); - - Job job = client.projects().locations().jobs().get(project, region, jobId).execute(); - LOG.info("Received job on get request for {}:\n{}", jobId, job.toPrettyString()); - return handleJobState(job); - } - - @Override - public void cancelJob(String project, String region, String jobId) - throws IOException { - LOG.info("Cancelling {} under {}", jobId, project); - Job job = new Job().setRequestedState(JobState.CANCELLED.toString()); - LOG.info("Sending job to update {}:\n{}", jobId, job.toPrettyString()); - client.projects().locations().jobs().update(project, region, jobId, job).execute(); - } - - private static JobState handleJobState(Job job) { - String currentState = job.getCurrentState(); - return Strings.isNullOrEmpty(currentState) ? JobState.UNKNOWN : JobState.parse(currentState); - } - - /** Builder for {@link FlexTemplateSdkClient}. */ + /** Builder for {@link FlexTemplateClient}. */ public static final class Builder { private Credentials credentials; @@ -118,8 +94,8 @@ public Builder setCredentials(Credentials value) { return this; } - public FlexTemplateSdkClient build() { - return new FlexTemplateSdkClient(this); + public FlexTemplateClient build() { + return new FlexTemplateClient(this); } } } diff --git a/it/src/main/java/com/google/cloud/teleport/it/dataflow/package-info.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/package-info.java new file mode 100644 index 0000000000..e5b5394f77 --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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 for managing Dataflow jobs from integration tests. */ +package com.google.cloud.teleport.it.dataflow; diff --git a/it/src/main/java/com/google/cloud/teleport/it/logging/LogStrings.java b/it/src/main/java/com/google/cloud/teleport/it/logging/LogStrings.java new file mode 100644 index 0000000000..f2ccac3331 --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/logging/LogStrings.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2022 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.it.logging; + +import com.google.api.client.json.GenericJson; +import com.google.api.services.dataflow.model.Job; +import com.google.common.collect.ImmutableMap; +import com.google.gson.GsonBuilder; +import java.util.Map; + +/** Utility for formatting different objects for easier readability in logs. */ +public final class LogStrings { + private LogStrings() {} + + /** Formats a Google API's {@link GenericJson} for pretty logging. */ + public static String formatForLogging(GenericJson genericJson) { + return formatForLogging(ImmutableMap.copyOf(genericJson)); + } + + /** + * Formats a Dataflow {@link Job} for pretty logging. + * + *

Some information will be excluded from the logs in order to improve readability and avoid + * hitting log limits. + */ + public static String formatForLogging(Job job) { + // The environment and steps can really pollute the logging output, making it hard to read + // and potentially causing problems on systems with limits to how much logging is allowed. + Job simpleCopy = + new Job() + .setId(job.getId()) + .setName(job.getName()) + .setProjectId(job.getProjectId()) + .setLocation(job.getLocation()) + .setCreateTime(job.getCreateTime()) + .setCurrentStateTime(job.getCurrentStateTime()) + .setRequestedState(job.getRequestedState()) // For when we try to cancel it + .setCurrentState(job.getCurrentState()) + .setLabels(job.getLabels()) + .setJobMetadata(job.getJobMetadata()); + return formatForLogging(ImmutableMap.copyOf(simpleCopy)); + } + + /** Formats a map for pretty logging. */ + public static String formatForLogging(Map map) { + return new GsonBuilder().setPrettyPrinting().create().toJson(map); + } +} diff --git a/it/src/main/java/com/google/cloud/teleport/it/logging/package-info.java b/it/src/main/java/com/google/cloud/teleport/it/logging/package-info.java new file mode 100644 index 0000000000..cbca92e0ab --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/logging/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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 for utilities to help make outputting readable logs easier. */ +package com.google.cloud.teleport.it.logging; diff --git a/it/src/main/java/com/google/cloud/teleport/it/package-info.java b/it/src/main/java/com/google/cloud/teleport/it/package-info.java new file mode 100644 index 0000000000..1f5eb81509 --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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 for general-purpose testing utilities. */ +package com.google.cloud.teleport.it; diff --git a/v2/testing/src/main/resources/test-artifact.txt b/it/src/main/resources/test-artifact.txt similarity index 100% rename from v2/testing/src/main/resources/test-artifact.txt rename to it/src/main/resources/test-artifact.txt diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java b/it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java similarity index 90% rename from v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java rename to it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java index 5d77151013..89d2a5b7c0 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/TestPropertiesTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java @@ -13,9 +13,10 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing; +package com.google.cloud.teleport.it; import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; import org.junit.After; import org.junit.Test; @@ -57,34 +58,34 @@ public void testAllPropertiesSet() { assertThat(properties.specPath()).isEqualTo(SPEC_PATH); } - @Test(expected = IllegalStateException.class) + @Test public void testAccessTokenNotSet() { System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - properties.accessToken(); + assertThrows(IllegalStateException.class, properties::accessToken); } - @Test(expected = IllegalStateException.class) + @Test public void testArtifactBucketNotSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - properties.artifactBucket(); + assertThrows(IllegalStateException.class, properties::artifactBucket); } - @Test(expected = IllegalStateException.class) + @Test public void testProjectNotSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - properties.project(); + assertThrows(IllegalStateException.class, properties::project); } @Test @@ -97,13 +98,13 @@ public void testRegionNotSet() { assertThat(properties.region()).isEqualTo(TestProperties.DEFAULT_REGION); } - @Test(expected = IllegalStateException.class) + @Test public void testSpecPathNotSet() { System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.REGION_KEY, REGION); - properties.specPath(); + assertThrows(IllegalStateException.class, properties::specPath); } } diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java b/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClientTest.java similarity index 96% rename from v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java rename to it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClientTest.java index 5abb563387..2cfa69e2b8 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactGcsSdkClientTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClientTest.java @@ -13,9 +13,10 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.artifacts; +package com.google.cloud.teleport.it.artifacts; import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyIterable; import static org.mockito.ArgumentMatchers.anyString; @@ -32,13 +33,13 @@ import com.google.cloud.storage.Storage; import com.google.cloud.storage.Storage.BlobListOption; import com.google.cloud.storage.Storage.BucketListOption; +import com.google.common.collect.ImmutableList; import com.google.common.io.Resources; import com.google.re2j.Pattern; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -95,11 +96,15 @@ public void testUploadArtifact() throws IOException { assertThat(actualContents).isEqualTo(TEST_ARTIFACT_CONTENTS); } - @Test(expected = IOException.class) - public void testUploadArtifactInvalidLocalPath() throws IOException { + @Test + public void testUploadArtifactInvalidLocalPath() { when(client.create(any(BlobInfo.class), any())).thenReturn(blob); - new ArtifactGcsSdkClient(client) - .uploadArtifact(BUCKET, DIR_PATH, "/some/invalid/path/please/do/not/make/this/file/valid"); + assertThrows( + IOException.class, + () -> + new ArtifactGcsSdkClient(client) + .uploadArtifact( + BUCKET, DIR_PATH, "/some/invalid/path/please/do/not/make/this/file/valid")); } @Test diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java b/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java similarity index 72% rename from v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java rename to it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java index 0e6823a75e..424f368561 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/artifacts/ArtifactUtilsTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java @@ -13,14 +13,11 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.artifacts; +package com.google.cloud.teleport.it.artifacts; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createGcsClient; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestDirName; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestPath; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestSuiteDirPath; import static com.google.common.truth.Truth.assertThat; +import com.google.common.truth.Truth; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -36,13 +33,13 @@ public final class ArtifactUtilsTest { @Test public void testCreateTestDirName() { - assertThat(createTestDirName()).matches(TEST_DIR_REGEX); + Truth.assertThat(ArtifactUtils.createTestDirName()).matches(TEST_DIR_REGEX); } @Test public void testCreateTestSuiteDirPath() { String suiteName = "some-test-class"; - String path = createTestSuiteDirPath(suiteName); + String path = ArtifactUtils.createTestSuiteDirPath(suiteName); assertThat(path).matches(String.format("%s/%s", suiteName, TEST_DIR_REGEX)); } @@ -51,14 +48,14 @@ public void testCreateTestPath() { String suiteDirPath = "some/test/suite/dir"; String testName = "some-test"; - String path = createTestPath(suiteDirPath, testName); + String path = ArtifactUtils.createTestPath(suiteDirPath, testName); assertThat(path).matches(String.format("%s/%s", suiteDirPath, testName)); } @Test public void testCreateClientWithNullCredentials() { - createGcsClient(null); + ArtifactUtils.createGcsClient(null); // Just making sure that no exceptions are thrown } } diff --git a/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java new file mode 100644 index 0000000000..990ff25c20 --- /dev/null +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java @@ -0,0 +1,131 @@ +/* + * Copyright (C) 2022 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.it.dataflow; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.Dataflow.Projects.Locations; +import com.google.api.services.dataflow.Dataflow.Projects.Locations.Jobs.Get; +import com.google.api.services.dataflow.Dataflow.Projects.Locations.Jobs.Update; +import com.google.api.services.dataflow.model.Job; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; +import java.io.IOException; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Answers; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +/** Unit tests for {@link AbstractDataflowTemplateClient}. */ +@RunWith(JUnit4.class) +public final class AbstractDataflowTemplateClientTest { + @Rule public final MockitoRule mockito = MockitoJUnit.rule(); + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Dataflow client; + + private static final String PROJECT = "test-project"; + private static final String REGION = "us-east1"; + private static final String JOB_ID = "test-job-id"; + + @Captor private ArgumentCaptor projectCaptor; + @Captor private ArgumentCaptor regionCaptor; + @Captor private ArgumentCaptor jobIdCaptor; + @Captor private ArgumentCaptor jobCaptor; + + @Test + public void testGetJobStatus() throws IOException { + Get get = mock(Get.class); + Job job = new Job().setCurrentState(JobState.RUNNING.toString()); + when(getLocationJobs(client).get(any(), any(), any())).thenReturn(get); + when(get.execute()).thenReturn(job); + + JobState actual = new FakeDataflowTemplateClient(client).getJobStatus(PROJECT, REGION, JOB_ID); + + verify(getLocationJobs(client)) + .get(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); + assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + assertThat(actual).isEqualTo(JobState.RUNNING); + } + + @Test + public void testGetJobThrowsException() throws IOException { + when(getLocationJobs(client).get(any(), any(), any())).thenThrow(new IOException()); + assertThrows( + IOException.class, + () -> new FakeDataflowTemplateClient(client).getJobStatus(PROJECT, REGION, JOB_ID)); + } + + @Test + public void testCancelJob() throws IOException { + Update update = mock(Update.class); + when(getLocationJobs(client).update(any(), any(), any(), any())).thenReturn(update); + when(update.execute()).thenReturn(new Job()); + + new FakeDataflowTemplateClient(client).cancelJob(PROJECT, REGION, JOB_ID); + + verify(getLocationJobs(client)) + .update( + projectCaptor.capture(), + regionCaptor.capture(), + jobIdCaptor.capture(), + jobCaptor.capture()); + assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); + assertThat(regionCaptor.getValue()).isEqualTo(REGION); + assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); + assertThat(jobCaptor.getValue().getRequestedState()).isEqualTo(JobState.CANCELLED.toString()); + } + + @Test + public void testCancelJobThrowsException() throws IOException { + when(getLocationJobs(client).update(any(), any(), any(), any())).thenThrow(new IOException()); + assertThrows( + IOException.class, + () -> new FakeDataflowTemplateClient(client).cancelJob(PROJECT, REGION, JOB_ID)); + } + + private static Locations.Jobs getLocationJobs(Dataflow client) { + return client.projects().locations().jobs(); + } + + /** + * Fake implementation that simply throws {@link UnsupportedOperationException} for some methods. + */ + private static final class FakeDataflowTemplateClient extends AbstractDataflowTemplateClient { + FakeDataflowTemplateClient(Dataflow client) { + super(client); + } + + @Override + public JobInfo launchTemplate(String project, String region, LaunchOptions options) + throws IOException { + throw new UnsupportedOperationException(); + } + } +} diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java similarity index 84% rename from v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java rename to it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java index e8f05b3e6c..003247017a 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowOperationTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; import static com.google.common.truth.Truth.assertThat; import static org.mockito.ArgumentMatchers.any; @@ -25,9 +25,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation.Config; -import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation.Result; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; +import com.google.cloud.teleport.it.dataflow.DataflowOperator.Config; +import com.google.cloud.teleport.it.dataflow.DataflowOperator.Result; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; import java.io.IOException; import java.time.Duration; import java.util.HashSet; @@ -44,12 +44,12 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; -/** Unit tests for {@link DataflowOperation}. */ +/** Unit tests for {@link DataflowOperator}. */ @RunWith(JUnit4.class) -public final class DataflowOperationTest { +public final class DataflowOperatorTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); - @Mock private FlexTemplateClient client; + @Mock private DataflowTemplateClient client; private static final String PROJECT = "test-project"; private static final String REGION = "us-east1"; @@ -80,7 +80,7 @@ public void testWaitUntilDone() throws IOException { .thenReturn(JobState.CANCELLED); // Act - Result result = DataflowOperation.waitUntilDone(client, DEFAULT_CONFIG); + Result result = new DataflowOperator(client).waitUntilDone(DEFAULT_CONFIG); // Assert verify(client, times(4)) @@ -99,7 +99,7 @@ public void testWaitUntilDone() throws IOException { @Test public void testWaitUntilDoneTimeout() throws IOException { when(client.getJobStatus(any(), any(), any())).thenReturn(JobState.RUNNING); - Result result = DataflowOperation.waitUntilDone(client, DEFAULT_CONFIG); + Result result = new DataflowOperator(client).waitUntilDone(DEFAULT_CONFIG); assertThat(result).isEqualTo(Result.TIMEOUT); } @@ -113,7 +113,7 @@ public void testWaitForCondition() throws IOException { .thenThrow(new IOException()) .thenReturn(JobState.RUNNING); - Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, checker); + Result result = new DataflowOperator(client).waitForCondition(DEFAULT_CONFIG, checker); verify(client, atMost(totalCalls)) .getJobStatus(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); @@ -129,7 +129,7 @@ public void testWaitForConditionJobFinished() throws IOException { .thenReturn(JobState.RUNNING) .thenReturn(JobState.CANCELLED); - Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + Result result = new DataflowOperator(client).waitForCondition(DEFAULT_CONFIG, () -> false); assertThat(result).isEqualTo(Result.JOB_FINISHED); } @@ -138,7 +138,7 @@ public void testWaitForConditionJobFinished() throws IOException { public void testWaitForConditionTimeout() throws IOException { when(client.getJobStatus(any(), any(), any())).thenReturn(JobState.RUNNING); - Result result = DataflowOperation.waitForCondition(client, DEFAULT_CONFIG, () -> false); + Result result = new DataflowOperator(client).waitForCondition(DEFAULT_CONFIG, () -> false); assertThat(result).isEqualTo(Result.TIMEOUT); } @@ -159,16 +159,13 @@ public void testFinishAfterCondition() throws IOException { doAnswer(invocation -> null).when(client).cancelJob(any(), any(), any()); // Act - Result result = DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, checker); + Result result = new DataflowOperator(client).waitForConditionAndFinish(DEFAULT_CONFIG, checker); // Assert verify(client, atLeast(totalCalls)) .getJobStatus(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); verify(client) - .cancelJob( - projectCaptor.capture(), - regionCaptor.capture(), - jobIdCaptor.capture()); + .cancelJob(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); Set allProjects = new HashSet<>(projectCaptor.getAllValues()); Set allRegions = new HashSet<>(regionCaptor.getAllValues()); @@ -188,7 +185,7 @@ public void testFinishAfterConditionJobStopped() throws IOException { doAnswer(invocation -> null).when(client).cancelJob(any(), any(), any()); Result result = - DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, () -> false); + new DataflowOperator(client).waitForConditionAndFinish(DEFAULT_CONFIG, () -> false); verify(client, never()).cancelJob(any(), any(), any()); assertThat(result).isEqualTo(Result.JOB_FINISHED); @@ -200,7 +197,7 @@ public void testFinishAfterConditionTimeout() throws IOException { doAnswer(invocation -> null).when(client).cancelJob(any(), any(), any()); Result result = - DataflowOperation.waitForConditionAndFinish(client, DEFAULT_CONFIG, () -> false); + new DataflowOperator(client).waitForConditionAndFinish(DEFAULT_CONFIG, () -> false); verify(client).cancelJob(any(), any(), any()); assertThat(result).isEqualTo(Result.TIMEOUT); diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java similarity index 87% rename from v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java rename to it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java index 0384086f26..f02d6c6ba2 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/DataflowUtilsTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java @@ -13,9 +13,9 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; -import static com.google.cloud.teleport.v2.testing.dataflow.DataflowUtils.createJobName; +import static com.google.cloud.teleport.it.dataflow.DataflowUtils.createJobName; import static com.google.common.truth.Truth.assertThat; import org.junit.Test; diff --git a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java similarity index 63% rename from v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java rename to it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java index ea2fafd312..c3c93d7700 100644 --- a/v2/testing/src/test/java/com/google/cloud/teleport/v2/testing/dataflow/FlexTemplateSdkClientTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java @@ -13,9 +13,10 @@ * License for the specific language governing permissions and limitations under * the License. */ -package com.google.cloud.teleport.v2.testing.dataflow; +package com.google.cloud.teleport.it.dataflow; import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -26,15 +27,14 @@ import com.google.api.services.dataflow.Dataflow.Projects.Locations.FlexTemplates; import com.google.api.services.dataflow.Dataflow.Projects.Locations.FlexTemplates.Launch; import com.google.api.services.dataflow.Dataflow.Projects.Locations.Jobs.Get; -import com.google.api.services.dataflow.Dataflow.Projects.Locations.Jobs.Update; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.LaunchFlexTemplateParameter; import com.google.api.services.dataflow.model.LaunchFlexTemplateRequest; import com.google.api.services.dataflow.model.LaunchFlexTemplateResponse; import com.google.auth.Credentials; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobInfo; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.LaunchOptions; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobInfo; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.LaunchOptions; import com.google.common.collect.ImmutableMap; import java.io.IOException; import org.junit.Rule; @@ -48,9 +48,9 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; -/** Unit test for {@link FlexTemplateSdkClient}. */ +/** Unit test for {@link FlexTemplateClient}. */ @RunWith(JUnit4.class) -public final class FlexTemplateSdkClientTest { +public final class FlexTemplateClientTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); @Mock(answer = Answers.RETURNS_DEEP_STUBS) @@ -69,12 +69,11 @@ public final class FlexTemplateSdkClientTest { @Captor private ArgumentCaptor regionCaptor; @Captor private ArgumentCaptor jobIdCaptor; @Captor private ArgumentCaptor requestCaptor; - @Captor private ArgumentCaptor jobCaptor; @Test public void testCreateWithCredentials() { Credentials credentials = mock(Credentials.class); - FlexTemplateSdkClient.builder().setCredentials(credentials).build(); + FlexTemplateClient.builder().setCredentials(credentials).build(); // Lack of exception is all we really can test } @@ -97,7 +96,7 @@ public void testLaunchNewJob() throws IOException { // Act JobInfo actual = - FlexTemplateSdkClient.withDataflowClient(client).launchNewJob(PROJECT, REGION, options); + FlexTemplateClient.withDataflowClient(client).launchTemplate(PROJECT, REGION, options); // Assert LaunchFlexTemplateRequest expectedRequest = @@ -123,61 +122,15 @@ public void testLaunchNewJob() throws IOException { assertThat(actual).isEqualTo(expected); } - @Test(expected = IOException.class) + @Test public void testLaunchNewJobThrowsException() throws IOException { when(getFlexTemplates(client).launch(any(), any(), any())).thenThrow(new IOException()); - FlexTemplateSdkClient.withDataflowClient(client) - .launchNewJob(PROJECT, REGION, LaunchOptions.builder(JOB_NAME, SPEC_PATH).build()); - } - - @Test - public void testGetJobStatus() throws IOException { - Get get = mock(Get.class); - Job job = new Job().setCurrentState(JobState.RUNNING.toString()); - when(getLocationJobs(client).get(any(), any(), any())).thenReturn(get); - when(get.execute()).thenReturn(job); - - JobState actual = - FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, REGION, JOB_ID); - - verify(getLocationJobs(client)) - .get(projectCaptor.capture(), regionCaptor.capture(), jobIdCaptor.capture()); - assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); - assertThat(regionCaptor.getValue()).isEqualTo(REGION); - assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); - assertThat(actual).isEqualTo(JobState.RUNNING); - } - - @Test(expected = IOException.class) - public void testGetJobThrowsException() throws IOException { - when(getLocationJobs(client).get(any(), any(), any())).thenThrow(new IOException()); - FlexTemplateSdkClient.withDataflowClient(client).getJobStatus(PROJECT, REGION, JOB_ID); - } - - @Test - public void testCancelJob() throws IOException { - Update update = mock(Update.class); - when(getLocationJobs(client).update(any(), any(), any(), any())).thenReturn(update); - when(update.execute()).thenReturn(new Job()); - - FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, REGION, JOB_ID); - - verify(getLocationJobs(client)) - .update( - projectCaptor.capture(), - regionCaptor.capture(), - jobIdCaptor.capture(), - jobCaptor.capture()); - assertThat(projectCaptor.getValue()).isEqualTo(PROJECT); - assertThat(regionCaptor.getValue()).isEqualTo(REGION); - assertThat(jobIdCaptor.getValue()).isEqualTo(JOB_ID); - assertThat(jobCaptor.getValue().getRequestedState()).isEqualTo(JobState.CANCELLED.toString()); - } - - @Test(expected = IOException.class) - public void testCancelJobThrowsException() throws IOException { - when(getLocationJobs(client).update(any(), any(), any(), any())).thenThrow(new IOException()); - FlexTemplateSdkClient.withDataflowClient(client).cancelJob(PROJECT, REGION, JOB_ID); + assertThrows( + IOException.class, + () -> + FlexTemplateClient.withDataflowClient(client) + .launchTemplate( + PROJECT, REGION, LaunchOptions.builder(JOB_NAME, SPEC_PATH).build())); } private static Locations.Jobs getLocationJobs(Dataflow client) { diff --git a/v2/pom.xml b/v2/pom.xml index 6d927cbd8b..9fd5502901 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -149,6 +149,12 @@ ${mockito-core.version} test + + com.google.cloud.teleport + it + 1.0-SNAPSHOT + test + @@ -679,7 +685,6 @@ streaming-data-generator kafka-to-pubsub kafka-common - testing diff --git a/v2/streaming-data-generator/pom.xml b/v2/streaming-data-generator/pom.xml index 3d1e0769cd..813a226298 100644 --- a/v2/streaming-data-generator/pom.xml +++ b/v2/streaming-data-generator/pom.xml @@ -71,12 +71,6 @@ ${truth.version} test - - com.google.cloud.teleport.v2 - testing - 1.0-SNAPSHOT - test - diff --git a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java index 444ac4bf9f..8aea312424 100644 --- a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java +++ b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java @@ -15,25 +15,25 @@ */ package com.google.cloud.teleport.v2.templates; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createGcsClient; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestPath; -import static com.google.cloud.teleport.v2.testing.artifacts.ArtifactUtils.createTestSuiteDirPath; -import static com.google.cloud.teleport.v2.testing.dataflow.DataflowUtils.createJobName; +import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createGcsClient; +import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createTestPath; +import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createTestSuiteDirPath; +import static com.google.cloud.teleport.it.dataflow.DataflowUtils.createJobName; import static com.google.common.truth.Truth.assertThat; import com.google.cloud.storage.Blob; import com.google.cloud.storage.Storage; +import com.google.cloud.teleport.it.TestProperties; +import com.google.cloud.teleport.it.artifacts.ArtifactClient; +import com.google.cloud.teleport.it.artifacts.ArtifactGcsSdkClient; +import com.google.cloud.teleport.it.dataflow.DataflowOperator; +import com.google.cloud.teleport.it.dataflow.DataflowOperator.Result; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobInfo; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.LaunchOptions; +import com.google.cloud.teleport.it.dataflow.FlexTemplateClient; import com.google.cloud.teleport.v2.templates.StreamingDataGenerator.SinkType; -import com.google.cloud.teleport.v2.testing.TestProperties; -import com.google.cloud.teleport.v2.testing.artifacts.ArtifactClient; -import com.google.cloud.teleport.v2.testing.artifacts.ArtifactGcsSdkClient; -import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation; -import com.google.cloud.teleport.v2.testing.dataflow.DataflowOperation.Result; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobInfo; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.JobState; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateClient.LaunchOptions; -import com.google.cloud.teleport.v2.testing.dataflow.FlexTemplateSdkClient; import com.google.common.io.Resources; import com.google.re2j.Pattern; import java.io.IOException; @@ -99,27 +99,27 @@ public void testFakeMessagesToGcs() throws IOException { String.format("gs://%s/%s", PROPERTIES.artifactBucket(), outputDir)) .addParameter(NUM_SHARDS_KEY, "1") .build(); - FlexTemplateClient dataflow = - FlexTemplateSdkClient.builder().setCredentials(PROPERTIES.googleCredentials()).build(); + DataflowTemplateClient dataflow = + FlexTemplateClient.builder().setCredentials(PROPERTIES.googleCredentials()).build(); - JobInfo info = dataflow.launchNewJob(PROPERTIES.project(), PROPERTIES.region(), options); + JobInfo info = dataflow.launchTemplate(PROPERTIES.project(), PROPERTIES.region(), options); assertThat(info.state()).isIn(JobState.RUNNING_STATES); Result result = - DataflowOperation.waitForConditionAndFinish( - dataflow, - createConfig(info), - () -> { - List outputFiles = - artifactClient.listArtifacts( - PROPERTIES.artifactBucket(), outputDir, Pattern.compile(".*output-.*")); - return !outputFiles.isEmpty(); - }); + new DataflowOperator(dataflow) + .waitForConditionAndFinish( + createConfig(info), + () -> { + List outputFiles = + artifactClient.listArtifacts( + PROPERTIES.artifactBucket(), outputDir, Pattern.compile(".*output-.*")); + return !outputFiles.isEmpty(); + }); assertThat(result).isEqualTo(Result.CONDITION_MET); } - private static DataflowOperation.Config createConfig(JobInfo info) { - return DataflowOperation.Config.builder() + private static DataflowOperator.Config createConfig(JobInfo info) { + return DataflowOperator.Config.builder() .setJobId(info.jobId()) .setProject(PROPERTIES.project()) .setRegion(PROPERTIES.region()) diff --git a/v2/testing/pom.xml b/v2/testing/pom.xml deleted file mode 100644 index e15fedb47f..0000000000 --- a/v2/testing/pom.xml +++ /dev/null @@ -1,33 +0,0 @@ - - - testing - 4.0.0 - - - dynamic-templates - com.google.cloud.teleport.v2 - 1.0-SNAPSHOT - - - - 1.0.1 - - - - - com.google.cloud - google-cloud-storage - - - - - com.google.truth - truth - ${truth.version} - test - - - - \ No newline at end of file diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java deleted file mode 100644 index 7c70947d4a..0000000000 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/artifacts/package-info.java +++ /dev/null @@ -1,2 +0,0 @@ -/** Package for working with test artifacts. */ -package com.google.cloud.teleport.v2.testing.artifacts; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java deleted file mode 100644 index 8ab5787140..0000000000 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/dataflow/package-info.java +++ /dev/null @@ -1,2 +0,0 @@ -/** Package for managing Dataflow jobs from integration tests. */ -package com.google.cloud.teleport.v2.testing.dataflow; diff --git a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java b/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java deleted file mode 100644 index 5f78495f17..0000000000 --- a/v2/testing/src/main/java/com/google/cloud/teleport/v2/testing/package-info.java +++ /dev/null @@ -1,2 +0,0 @@ -/** Package for general-purpose testing utilities. */ -package com.google.cloud.teleport.v2.testing; From a2658747ea7994e8660dc127be8e601f60a761c3 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 27 Jan 2022 15:14:07 -0500 Subject: [PATCH 071/145] Add a README to it/ to make it easier to understand what it is for --- it/README.md | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 it/README.md diff --git a/it/README.md b/it/README.md new file mode 100644 index 0000000000..944265728d --- /dev/null +++ b/it/README.md @@ -0,0 +1,11 @@ +# Integration Test Utilities + +## What Is Included + +Anything that can be used to aid writing integration tests for either Classic +or Flex Templates. + +## What Is Not Included + +The integration tests themselves. Those should go in the test folder for the +template that they are being written for. From be043d23cfdf2c4d32fc2d2e4ff5f9c6ef7b876e Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 27 Jan 2022 15:59:34 -0500 Subject: [PATCH 072/145] Make necessary changes to get the integration test working --- cicd/internal/repo/modules.go | 1 + cicd/internal/repo/modules_test.go | 2 +- it/pom.xml | 2 +- .../cloud/teleport/it/TestProperties.java | 108 +++++----- .../cloud/teleport/it/artifacts/Artifact.java | 34 ++++ .../teleport/it/artifacts/ArtifactClient.java | 71 +++++-- .../it/artifacts/ArtifactGcsSdkClient.java | 118 ----------- .../teleport/it/artifacts/ArtifactUtils.java | 40 ++-- .../teleport/it/artifacts/GcsArtifact.java | 43 ++++ .../it/artifacts/GcsArtifactClient.java | 192 ++++++++++++++++++ .../it/dataflow/DataflowTemplateClient.java | 14 +- .../it/dataflow/FlexTemplateClient.java | 2 +- .../cloud/teleport/it/TestPropertiesTest.java | 36 +--- .../it/artifacts/ArtifactUtilsTest.java | 35 ++-- ...ntTest.java => GcsArtifactClientTest.java} | 144 ++++++++----- .../it/artifacts/GcsArtifactTest.java | 64 ++++++ .../AbstractDataflowTemplateClientTest.java | 2 +- .../it/dataflow/FlexTemplateClientTest.java | 8 +- v2/pom.xml | 2 +- .../templates/StreamingDataGeneratorIT.java | 68 ++++--- 20 files changed, 637 insertions(+), 349 deletions(-) create mode 100644 it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java delete mode 100644 it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java create mode 100644 it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifact.java create mode 100644 it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClient.java rename it/src/test/java/com/google/cloud/teleport/it/artifacts/{ArtifactGcsSdkClientTest.java => GcsArtifactClientTest.java} (67%) create mode 100644 it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactTest.java diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index c3bc477fd6..3576963cfb 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -34,6 +34,7 @@ const ( func GetAllRoots() []string { return []string{ ClassicRoot, + ItRoot, FlexRoot, } } diff --git a/cicd/internal/repo/modules_test.go b/cicd/internal/repo/modules_test.go index b4185dd9b6..ad70e3c030 100644 --- a/cicd/internal/repo/modules_test.go +++ b/cicd/internal/repo/modules_test.go @@ -23,7 +23,7 @@ import ( func TestGetAllRoots(t *testing.T) { actual := GetAllRoots() - expected := []string{ClassicRoot, FlexRoot} + expected := []string{ClassicRoot, ItRoot, FlexRoot} if !reflect.DeepEqual(actual, expected) { t.Errorf("Not all roots present. Expected: %v. Got: %v", expected, actual) } diff --git a/it/pom.xml b/it/pom.xml index 771d20d469..cd32dede75 100644 --- a/it/pom.xml +++ b/it/pom.xml @@ -17,7 +17,7 @@ com.google.cloud.teleport - it + integration-testing-lib 1.0-SNAPSHOT 4.0.0 diff --git a/it/src/main/java/com/google/cloud/teleport/it/TestProperties.java b/it/src/main/java/com/google/cloud/teleport/it/TestProperties.java index c0b0451d60..d98bc0f414 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/TestProperties.java +++ b/it/src/main/java/com/google/cloud/teleport/it/TestProperties.java @@ -20,87 +20,81 @@ import com.google.auth.Credentials; import com.google.auth.oauth2.AccessToken; import com.google.auth.oauth2.GoogleCredentials; -import com.google.common.base.Strings; -import java.util.HashMap; -import java.util.Map; +import javax.annotation.Nullable; /** * Utility for accessing system properties set for the test. * - *

The values should be passed to the test like `-Dkey=value`. For instance, - * `-Dproject=my-project`. + *

There are two types of properties: those set on the command lines and those set as environment + * variables. Those set on the command line always follow a camelCase naming convention, and those + * set as environment variable always follow a CAPITALIZED_SNAKE_CASE naming convention. */ public final class TestProperties { - public static final String ACCESS_TOKEN_KEY = "accessToken"; + private TestProperties() {} + + // For testability, it is normally best to expect each property from the command line. We should + // only expect an environment variable if we're trying to avoid an accidental log of the + // value. + + // From command line public static final String ARTIFACT_BUCKET_KEY = "artifactBucket"; public static final String PROJECT_KEY = "project"; public static final String REGION_KEY = "region"; public static final String SPEC_PATH_KEY = "specPath"; + // From environment variables + public static final String ACCESS_TOKEN_KEY = "DT_IT_ACCESS_TOKEN"; + + // Default values for optional properties public static final String DEFAULT_REGION = "us-central1"; - private static String accessToken; - private static String artifactBucket; - private static String project; - private static String region; - private static String specPath; - - private final Map initialized; - - public TestProperties() { - initialized = new HashMap<>(); - initialized.put(ACCESS_TOKEN_KEY, false); - initialized.put(ARTIFACT_BUCKET_KEY, false); - initialized.put(PROJECT_KEY, false); - initialized.put(REGION_KEY, false); - initialized.put(SPEC_PATH_KEY, false); - } + // Error messages + private static final String CLI_ERR_MSG = "-D%s is required on the command line"; + private static final String ENV_VAR_MSG = "%s is required as an environment variable"; - public String accessToken() { - if (!initialized.get(ACCESS_TOKEN_KEY)) { - accessToken = System.getProperty(ACCESS_TOKEN_KEY, null); - checkState(!Strings.isNullOrEmpty(accessToken), "%s is required", ACCESS_TOKEN_KEY); - initialized.replace(ACCESS_TOKEN_KEY, true); - } - return accessToken; + public static String accessToken() { + return getProperty(ACCESS_TOKEN_KEY, Type.ENVIRONMENT_VARIABLE); } - public Credentials googleCredentials() { + public static Credentials googleCredentials() { return new GoogleCredentials(new AccessToken(accessToken(), /* expirationTime= */ null)); } - public String artifactBucket() { - if (!initialized.get(ARTIFACT_BUCKET_KEY)) { - artifactBucket = System.getProperty(ARTIFACT_BUCKET_KEY, null); - checkState(!Strings.isNullOrEmpty(artifactBucket), "%s is required", ARTIFACT_BUCKET_KEY); - initialized.replace(ARTIFACT_BUCKET_KEY, true); - } - return artifactBucket; + public static String artifactBucket() { + return getProperty(ARTIFACT_BUCKET_KEY, Type.PROPERTY); + } + + public static String project() { + return getProperty(PROJECT_KEY, Type.PROPERTY); } - public String project() { - if (!initialized.get(PROJECT_KEY)) { - project = System.getProperty(PROJECT_KEY, null); - checkState(!Strings.isNullOrEmpty(project), "%s is required", PROJECT_KEY); - initialized.replace(PROJECT_KEY, true); - } - return project; + public static String region() { + return getProperty(REGION_KEY, DEFAULT_REGION, Type.PROPERTY); + } + + public static String specPath() { + return getProperty(SPEC_PATH_KEY, Type.PROPERTY); + } + + /** Gets a property or throws an exception if it is not found. */ + private static String getProperty(String name, Type type) { + String value = getProperty(name, null, type); + String errMsg = + type == Type.PROPERTY ? String.format(CLI_ERR_MSG, name) : String.format(ENV_VAR_MSG, name); + + checkState(value != null, errMsg); + return value; } - public String region() { - if (!initialized.get(REGION_KEY)) { - region = System.getProperty(REGION_KEY, DEFAULT_REGION); - initialized.replace(REGION_KEY, true); - } - return region; + /** Gets a property or returns {@code defaultValue} if it is not found. */ + private static String getProperty(String name, @Nullable String defaultValue, Type type) { + String value = type == Type.PROPERTY ? System.getProperty(name) : System.getenv(name); + return value != null ? value : defaultValue; } - public String specPath() { - if (!initialized.get(SPEC_PATH_KEY)) { - specPath = System.getProperty(SPEC_PATH_KEY, null); - checkState(!Strings.isNullOrEmpty(specPath), "%s is required", SPEC_PATH_KEY); - initialized.replace(SPEC_PATH_KEY, true); - } - return specPath; + /** Defines the types of properties there may be. */ + private enum Type { + PROPERTY, + ENVIRONMENT_VARIABLE } } diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java new file mode 100644 index 0000000000..45c3e8f238 --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java @@ -0,0 +1,34 @@ +/* + * Copyright (C) 2022 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.it.artifacts; + +/** + * Represents a single artifact. + * + *

Implementations should remain read-only. Writing artifacts should be left to the + * responsibility of a {@link ArtifactClient} implementation. If an object of the artifact type + * returned allows writing of any type, then it should not be made available. + */ +public interface Artifact { + /** Returns the id of the artifact. */ + String id(); + + /** Returns the name/path of the artifact. */ + String name(); + + /** Returns the raw byte array of the artifact's contents. */ + byte[] contents(); +} diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java index 66224260f7..80b6b21aae 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactClient.java @@ -15,39 +15,72 @@ */ package com.google.cloud.teleport.it.artifacts; -import com.google.cloud.storage.Blob; import com.google.re2j.Pattern; import java.io.IOException; import java.util.List; -/** Interface for working with test artifacts. */ +/** + * Interface for working with test artifacts. + * + *

It is the responsibility of implementations to make sure that artifacts are kept separate from + * each other. Using a GCS path, this isolation would create a path like the following: {@code + * gs://test-class-name/run-id/test-method-name}. Each directory means: + * + *

    + *
  • test-class-name: A name given to the directory for a test class. This does not need to be + * identical to the class name, but it should clearly identify the class from other test + * classes. This is intended for long-lived artifacts that have value beyond a specific run of + * a test, such as a result file. + *
  • run-id: An id assigned to a run of that test class. This will be handled by implementations + * of this client. It is intended for artifacts that may be referenced by multiple methods in + * a test class. + *
  • test-method-name: A name given to the directory for a method within the test class. This + * does not need to be identical to the method name, but it should clearly identify the method + * from other test methods within the same test class. This is intended for input and output + * artifacts specific to the test method. + *
+ * + *

Separate input/output directories are optional and the responsibility of the test writer to + * maintain. + */ public interface ArtifactClient { + + /** Returns the id associated with the particular run of the test class. */ + String runId(); + /** - * Uploads a local file to GCS. + * Creates a new artifact in whatever service is being used to store them. * - * @param bucket the GCS bucket to upload to - * @param gcsPath the path from the bucket root to upload to - * @param localPath the path to the local file - * @return the {@link Blob} that was created - * @throws IOException if the local file cannot be read + * @param artifactName the name of the artifact. If this is supposed to go under an input/output + * directory, then it should include that (example: input/artifact.txt) + * @param contents the contents of the artifact + * @return a representation of the created artifact */ - Blob uploadArtifact(String bucket, String gcsPath, String localPath) throws IOException; + Artifact createArtifact(String artifactName, byte[] contents); /** - * Lists all artifacts in the given directory that match a given regex. + * Uploads a local file to the service being used for storing artifacts. * - * @param bucket the bucket the artifacts are in - * @param testDirPath the directory in the bucket that the artifacts are in - * @param regex the regex to use for matching artifacts - * @return all the {@link Blob}s that match the regex + * @param artifactName the name of the artifact. If this is supposed to go under an input/output + * directory, then it should include that (example: input/artifact.txt) + * @param localPath the local path to the file to upload + * @return a representation of the uploaded artifact + * @throws IOException if there is an issue reading the local file */ - List listArtifacts(String bucket, String testDirPath, Pattern regex); + Artifact uploadArtifact(String artifactName, String localPath) throws IOException; + + // TODO(zhoufek): Add equivalents for the above for uploading artifacts of a test method /** - * Removes the directory from the bucket. + * Lists all artifacts under test-class-name/run-id/{@code prefix}. * - * @param bucket the bucket with the directory to remove - * @param testDirPath the directory to remove + * @param prefix the prefix to use along with the fixed values method above. This must include the + * test-method-name value, but it can include other directories or files under it. + * @param regex a regex to use for filtering out unwanted artifacts + * @return all the artifacts whose name matches regex */ - void deleteTestDir(String bucket, String testDirPath); + List listArtifacts(String prefix, Pattern regex); + + /** Deletes all the files located under test-class-name/run-id. */ + void cleanupRun(); } diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java deleted file mode 100644 index 62096fed0d..0000000000 --- a/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClient.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Copyright (C) 2022 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.it.artifacts; - -import static com.google.common.collect.ImmutableList.toImmutableList; - -import com.google.api.gax.paging.Page; -import com.google.cloud.storage.Blob; -import com.google.cloud.storage.BlobId; -import com.google.cloud.storage.BlobInfo; -import com.google.cloud.storage.Storage; -import com.google.cloud.storage.Storage.BlobListOption; -import com.google.common.collect.ImmutableList; -import com.google.re2j.Pattern; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; -import java.util.function.Consumer; -import java.util.stream.StreamSupport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Client for working with test artifacts which uses the GCS SDK. */ -public final class ArtifactGcsSdkClient implements ArtifactClient { - private static final Logger LOG = LoggerFactory.getLogger(ArtifactGcsSdkClient.class); - - private final Storage client; - - public ArtifactGcsSdkClient(Storage client) { - this.client = client; - } - - @Override - public Blob uploadArtifact(String bucket, String gcsPath, String localPath) throws IOException { - LOG.info("Uploading {} to {} under {}", localPath, gcsPath, bucket); - BlobId id = BlobId.of(bucket, gcsPath); - BlobInfo info = BlobInfo.newBuilder(id).build(); - - byte[] contents = Files.readAllBytes(Paths.get(localPath)); - - return client.create(info, contents); - } - - @Override - public List listArtifacts(String bucket, String testDirPath, Pattern regex) { - List result = new ArrayList<>(); - consumeTestDir( - bucket, - testDirPath, - blobs -> { - for (Blob blob : blobs) { - if (regex.matches(blob.getName())) { - result.add(blob); - } - } - }); - return result; - } - - @Override - public void deleteTestDir(String bucket, String testDirPath) { - LOG.info("Deleting everything in {} under {}", testDirPath, bucket); - consumeTestDir( - bucket, - testDirPath, - blobs -> { - // Go through the Iterable overload, since the other ones make it very difficult - // to - // do thorough testing with Mockito - ImmutableList blobIds = - StreamSupport.stream(blobs.spliterator(), false) - .map(Blob::getBlobId) - .collect(toImmutableList()); - if (blobIds.isEmpty()) { - return; - } - List deleted = client.delete(blobIds); - for (int i = 0; i < blobIds.size(); ++i) { - if (!deleted.get(i)) { - LOG.warn("Blob {} not deleted", blobIds.get(i).getName()); - } - } - }); - } - - private void consumeTestDir( - String bucket, String testDirPath, Consumer> consumeBlobs) { - Page blobs = getFirstTestDirPage(bucket, testDirPath); - while (true) { - consumeBlobs.accept(blobs.getValues()); - - if (blobs.hasNextPage()) { - blobs = blobs.getNextPage(); - } else { - break; - } - } - } - - private Page getFirstTestDirPage(String bucket, String testDirPath) { - return client.list(bucket, BlobListOption.prefix(testDirPath)); - } -} diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java index 074fb75e12..53a955142c 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/ArtifactUtils.java @@ -15,6 +15,9 @@ */ package com.google.cloud.teleport.it.artifacts; +import static java.util.Arrays.stream; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + import com.google.auth.Credentials; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageOptions; @@ -22,14 +25,15 @@ import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.UUID; -import javax.annotation.Nullable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; /** Utilities for working with test artifacts. */ public final class ArtifactUtils { private ArtifactUtils() {} - /** Creates a unique name for the test directory. */ - public static String createTestDirName() { + /** Creates a unique id for the run. */ + public static String createRunId() { return String.format( "%s-%s", DateTimeFormatter.ofPattern("yyyyMMdd").withZone(ZoneId.of("UTC")).format(Instant.now()), @@ -37,25 +41,21 @@ public static String createTestDirName() { } /** - * Creates a unique path for the test suite. + * Returns the full GCS path given a list of path parts. * - * @param suiteDir the name of the test suite. This is generally the class with all the tests in - * it. - */ - public static String createTestSuiteDirPath(String suiteDir) { - return String.format("%s/%s", suiteDir, createTestDirName()); - } - - /** - * Creates a path for artifacts from an individual test to go into. + *

"path parts" refers to the bucket, directories, and file. Only the bucket is mandatory and + * must be the first value provided. * - * @param suiteDirPath the name of the test suite. This is generally the class with all the tests - * in it. - * @param testName the name of the test. It is the responsibility of the caller to make sure all - * their test names are unique. + * @param pathParts everything that makes up the path, minus the separators. There must be at + * least one value, and none of them can be empty + * @return the full path, such as 'gs://bucket/dir1/dir2/file' */ - public static String createTestPath(String suiteDirPath, String testName) { - return String.format("%s/%s", suiteDirPath, testName); + public static String getFullGcsPath(String... pathParts) { + checkArgument(pathParts.length != 0, "Must provide at least one path part"); + checkArgument( + stream(pathParts).noneMatch(Strings::isNullOrEmpty), "No path part can be null or empty"); + + return String.format("gs://%s", Joiner.on('/').join(pathParts)); } /** @@ -66,7 +66,7 @@ public static String createTestPath(String suiteDirPath, String testName) { * it is best to pass in a short-lived access token. * @return a {@link Storage} client for running GCS operations */ - public static Storage createGcsClient(@Nullable Credentials credentials) { + public static Storage createGcsClient(Credentials credentials) { StorageOptions.Builder builder = StorageOptions.newBuilder(); if (credentials != null) { builder.setCredentials(credentials); diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifact.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifact.java new file mode 100644 index 0000000000..31323b35bd --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifact.java @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2022 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.it.artifacts; + +import com.google.cloud.storage.Blob; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; + +/** Represents a single blob in GCS. */ +public final class GcsArtifact implements Artifact { + @VisibleForTesting final Blob blob; + + GcsArtifact(Blob blob) { + this.blob = blob; + } + + @Override + public String id() { + return blob.getGeneratedId(); + } + + @Override + public String name() { + return blob.getName(); + } + + @Override + public byte[] contents() { + return blob.getContent(); + } +} diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClient.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClient.java new file mode 100644 index 0000000000..92fd70ece3 --- /dev/null +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClient.java @@ -0,0 +1,192 @@ +/* + * Copyright (C) 2022 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.it.artifacts; + +import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createRunId; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.api.gax.paging.Page; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.Storage.BlobListOption; +import com.google.re2j.Pattern; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Client for working with test artifacts stored in Google Cloud Storage. + * + *

Tests should store this as a static value of the class and call {@link + * ArtifactClient#cleanupRun()} in the {@code @AfterClass} method. + */ +public final class GcsArtifactClient implements ArtifactClient { + private static final Logger LOG = LoggerFactory.getLogger(GcsArtifactClient.class); + + private final Storage client; + private final String bucket; + private final String testClassName; + private final String runId; + + public GcsArtifactClient(Builder builder) { + this.client = builder.client; + this.bucket = builder.bucket; + this.testClassName = builder.testClassName; + this.runId = createRunId(); + } + + /** Returns a new {@link Builder} for configuring a client. */ + public static Builder builder(Storage client, String bucket, String testClassName) { + checkArgument(!bucket.equals("")); + checkArgument(!testClassName.equals("")); + + return new Builder(client, bucket, testClassName); + } + + @Override + public String runId() { + return runId; + } + + @Override + public Artifact createArtifact(String artifactName, byte[] contents) { + String path = joinPathParts(testClassName, runId, artifactName); + return handleCreate(path, contents); + } + + @Override + public Artifact uploadArtifact(String artifactName, String localPath) throws IOException { + LOG.info( + "Uploading '{}' to file '{}' under '{}'", + localPath, + artifactName, + joinPathParts(testClassName, runId)); + return createArtifact(artifactName, Files.readAllBytes(Paths.get(localPath))); + } + + /** + * Helper for creating an artifact. + * + * @param path the full path under the bucket + * @param contents the contents of the artifact + * @return a representation of the artifact + */ + private Artifact handleCreate(String path, byte[] contents) { + LOG.info("Uploading {} bytes to '{}' under bucket '{}'", contents.length, path, bucket); + + BlobId id = BlobId.of(bucket, path); + BlobInfo info = BlobInfo.newBuilder(id).build(); + Blob blob = client.create(info, contents); + LOG.info( + "Successfully uploaded {} bytes to '{}' under bucket '{}'", contents.length, path, bucket); + + return new GcsArtifact(blob); + } + + @Override + public List listArtifacts(String prefix, Pattern regex) { + String listFrom = joinPathParts(testClassName, runId, prefix); + LOG.info("Listing everything under '{}' that matches '{}'", listFrom, regex.pattern()); + + List matched = new ArrayList<>(); + Page firstPage = getFirstPage(listFrom); + consumePages( + firstPage, + blobs -> { + for (Blob blob : blobs) { + if (regex.matches(blob.getName())) { + matched.add(new GcsArtifact(blob)); + } + } + }); + + return matched; + } + + @Override + public void cleanupRun() { + String path = joinPathParts(testClassName, runId); + LOG.info("Cleaning up everything under '{}' under bucket '{}'", path, bucket); + + Page firstPage = getFirstPage(path); + consumePages( + firstPage, + blobs -> { + // For testability, use the Iterable overload + List blobIds = new ArrayList<>(); + for (Blob blob : blobs) { + blobIds.add(blob.getBlobId()); + } + if (blobIds.isEmpty()) { + return; + } + + List deleted = client.delete(blobIds); + for (int i = 0; i < deleted.size(); ++i) { + if (!deleted.get(i)) { + LOG.warn("Blob '{}' not deleted", blobIds.get(i).getName()); + } + } + }); + } + + private void consumePages(Page firstPage, Consumer> consumeBlobs) { + Page currentPage = firstPage; + while (true) { + consumeBlobs.accept(currentPage.getValues()); + if (currentPage.hasNextPage()) { + currentPage = currentPage.getNextPage(); + } else { + break; + } + } + } + + private Page getFirstPage(String prefix) { + return client.list(bucket, BlobListOption.prefix(prefix)); + } + + private static String joinPathParts(String... parts) { + return Joiner.on('/').join(parts); + } + + /** Builder for {@link GcsArtifactClient}. */ + public static final class Builder { + private final Storage client; + private final String bucket; + private final String testClassName; + + private Builder(Storage client, String bucket, String testClassName) { + this.client = client; + this.bucket = bucket; + this.testClassName = testClassName; + } + + // TODO(zhoufek): Let users control the page size and other configurations + + public GcsArtifactClient build() { + return new GcsArtifactClient(this); + } + } +} diff --git a/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java index f228b61dd8..3b313dae24 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java @@ -76,13 +76,13 @@ public String toString() { } } - /** LaunchOptions for starting a Dataflow job. */ - class LaunchOptions { + /** Config for starting a Dataflow job. */ + class LaunchConfig { private final String jobName; private final ImmutableMap parameters; private final String specPath; - private LaunchOptions(Builder builder) { + private LaunchConfig(Builder builder) { this.jobName = builder.jobName; this.parameters = ImmutableMap.copyOf(builder.parameters); this.specPath = builder.specPath; @@ -104,7 +104,7 @@ public static Builder builder(String jobName, String specPath) { return new Builder(jobName, specPath); } - /** Builder for the {@link LaunchOptions}. */ + /** Builder for the {@link LaunchConfig}. */ public static final class Builder { private final String jobName; private final Map parameters; @@ -134,8 +134,8 @@ public String getSpecPath() { return specPath; } - public LaunchOptions build() { - return new LaunchOptions(this); + public LaunchConfig build() { + return new LaunchConfig(this); } } } @@ -171,7 +171,7 @@ public abstract static class Builder { * @return info about the request to launch a new job * @throws IOException if there is an issue sending the request */ - JobInfo launchTemplate(String project, String region, LaunchOptions options) throws IOException; + JobInfo launchTemplate(String project, String region, LaunchConfig options) throws IOException; /** * Gets the current status of a job. diff --git a/it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java index 596615a5dd..987e7f57d9 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClient.java @@ -54,7 +54,7 @@ public static Builder builder() { } @Override - public JobInfo launchTemplate(String project, String region, LaunchOptions options) + public JobInfo launchTemplate(String project, String region, LaunchConfig options) throws IOException { LOG.info("Getting ready to launch {} in {} under {}", options.jobName(), region, project); LOG.info("Using the spec at {}", options.specPath()); diff --git a/it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java b/it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java index 89d2a5b7c0..9e0c64996a 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/TestPropertiesTest.java @@ -26,17 +26,13 @@ /** Unit tests for {@link TestProperties}. */ @RunWith(JUnit4.class) public final class TestPropertiesTest { - private static final String ACCESS_TOKEN = "some-token"; private static final String ARTIFACT_BUCKET = "test-bucket"; private static final String PROJECT = "test-project"; private static final String REGION = "us-east1"; private static final String SPEC_PATH = "gs://test-bucket/some/spec/path"; - private final TestProperties properties = new TestProperties(); - @After public void tearDown() { - System.clearProperty(TestProperties.ACCESS_TOKEN_KEY); System.clearProperty(TestProperties.ARTIFACT_BUCKET_KEY); System.clearProperty(TestProperties.PROJECT_KEY); System.clearProperty(TestProperties.REGION_KEY); @@ -45,66 +41,50 @@ public void tearDown() { @Test public void testAllPropertiesSet() { - System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); - System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); - System.setProperty(TestProperties.PROJECT_KEY, PROJECT); - System.setProperty(TestProperties.REGION_KEY, REGION); - System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - - assertThat(properties.accessToken()).isEqualTo(ACCESS_TOKEN); - assertThat(properties.artifactBucket()).isEqualTo(ARTIFACT_BUCKET); - assertThat(properties.project()).isEqualTo(PROJECT); - assertThat(properties.region()).isEqualTo(REGION); - assertThat(properties.specPath()).isEqualTo(SPEC_PATH); - } - - @Test - public void testAccessTokenNotSet() { System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - assertThrows(IllegalStateException.class, properties::accessToken); + assertThat(TestProperties.artifactBucket()).isEqualTo(ARTIFACT_BUCKET); + assertThat(TestProperties.project()).isEqualTo(PROJECT); + assertThat(TestProperties.region()).isEqualTo(REGION); + assertThat(TestProperties.specPath()).isEqualTo(SPEC_PATH); } @Test public void testArtifactBucketNotSet() { - System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - assertThrows(IllegalStateException.class, properties::artifactBucket); + assertThrows(IllegalStateException.class, TestProperties::artifactBucket); } @Test public void testProjectNotSet() { - System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.REGION_KEY, REGION); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - assertThrows(IllegalStateException.class, properties::project); + assertThrows(IllegalStateException.class, TestProperties::project); } @Test public void testRegionNotSet() { - System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.SPEC_PATH_KEY, SPEC_PATH); - assertThat(properties.region()).isEqualTo(TestProperties.DEFAULT_REGION); + assertThat(TestProperties.region()).isEqualTo(TestProperties.DEFAULT_REGION); } @Test public void testSpecPathNotSet() { - System.setProperty(TestProperties.ACCESS_TOKEN_KEY, ACCESS_TOKEN); System.setProperty(TestProperties.ARTIFACT_BUCKET_KEY, ARTIFACT_BUCKET); System.setProperty(TestProperties.PROJECT_KEY, PROJECT); System.setProperty(TestProperties.REGION_KEY, REGION); - assertThrows(IllegalStateException.class, properties::specPath); + assertThrows(IllegalStateException.class, TestProperties::specPath); } } diff --git a/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java b/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java index 424f368561..6236c54033 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactUtilsTest.java @@ -16,8 +16,8 @@ package com.google.cloud.teleport.it.artifacts; import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; -import com.google.common.truth.Truth; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,29 +33,36 @@ public final class ArtifactUtilsTest { @Test public void testCreateTestDirName() { - Truth.assertThat(ArtifactUtils.createTestDirName()).matches(TEST_DIR_REGEX); + assertThat(ArtifactUtils.createRunId()).matches(TEST_DIR_REGEX); } @Test - public void testCreateTestSuiteDirPath() { - String suiteName = "some-test-class"; - String path = ArtifactUtils.createTestSuiteDirPath(suiteName); - assertThat(path).matches(String.format("%s/%s", suiteName, TEST_DIR_REGEX)); + public void testGetFullGcsPath() { + assertThat(ArtifactUtils.getFullGcsPath("bucket", "dir1", "dir2", "file")) + .isEqualTo("gs://bucket/dir1/dir2/file"); } @Test - public void testCreateTestPath() { - String suiteDirPath = "some/test/suite/dir"; - String testName = "some-test"; + public void testGetFullGcsPathOnlyBucket() { + assertThat(ArtifactUtils.getFullGcsPath("bucket")).isEqualTo("gs://bucket"); + } - String path = ArtifactUtils.createTestPath(suiteDirPath, testName); + @Test + public void testGetFullGcsPathEmpty() { + assertThrows(IllegalArgumentException.class, ArtifactUtils::getFullGcsPath); + } - assertThat(path).matches(String.format("%s/%s", suiteDirPath, testName)); + @Test + public void testGetFullGcsPathOneNullValue() { + assertThrows( + IllegalArgumentException.class, + () -> ArtifactUtils.getFullGcsPath("bucket", null, "dir2", "file")); } @Test - public void testCreateClientWithNullCredentials() { - ArtifactUtils.createGcsClient(null); - // Just making sure that no exceptions are thrown + public void testGetFullGcsPathOneEmptyValue() { + assertThrows( + IllegalArgumentException.class, + () -> ArtifactUtils.getFullGcsPath("bucket", "", "dir2", "file")); } } diff --git a/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClientTest.java b/it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClientTest.java similarity index 67% rename from it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClientTest.java rename to it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClientTest.java index 2cfa69e2b8..c6d805cf52 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/artifacts/ArtifactGcsSdkClientTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactClientTest.java @@ -40,6 +40,8 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; +import java.util.UUID; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -50,19 +52,21 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; -/** Unit tests for {@link ArtifactGcsSdkClient}. */ +/** Unit tests for {@link GcsArtifactClient}. */ @RunWith(JUnit4.class) -public final class ArtifactGcsSdkClientTest { +public final class GcsArtifactClientTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); @Mock private Storage client; @Mock private Blob blob; + private GcsArtifactClient artifactClient; + private static final String ARTIFACT_NAME = "test-artifact.txt"; private static final String LOCAL_PATH; private static final byte[] TEST_ARTIFACT_CONTENTS; static { - LOCAL_PATH = Resources.getResource("test-artifact.txt").getPath(); + LOCAL_PATH = Resources.getResource(ARTIFACT_NAME).getPath(); try { TEST_ARTIFACT_CONTENTS = Files.readAllBytes(Paths.get(LOCAL_PATH)); } catch (IOException e) { @@ -71,7 +75,8 @@ public final class ArtifactGcsSdkClientTest { } private static final String BUCKET = "test-bucket"; - private static final String DIR_PATH = "some/dir/path"; + private static final String TEST_CLASS = "test-class-name"; + private static final String TEST_METHOD = "test-method-name"; @Captor private ArgumentCaptor bucketCaptor; @Captor private ArgumentCaptor blobInfoCaptor; @@ -79,21 +84,57 @@ public final class ArtifactGcsSdkClientTest { @Captor private ArgumentCaptor listOptionsCaptor; @Captor private ArgumentCaptor> blobIdCaptor; + @Before + public void setUp() { + artifactClient = GcsArtifactClient.builder(client, BUCKET, TEST_CLASS).build(); + } + @Test - public void testUploadArtifact() throws IOException { + public void testBuilderWithEmptyBucket() { + assertThrows( + IllegalArgumentException.class, + () -> GcsArtifactClient.builder(client, "", TEST_CLASS).build()); + } + + @Test + public void testBuilderWithEmptyTestClassName() { + assertThrows( + IllegalArgumentException.class, + () -> GcsArtifactClient.builder(client, BUCKET, "").build()); + } + + @Test + public void testCreateArtifactInRunDir() { + String artifactName = "artifact.txt"; + byte[] contents = new byte[] {0, 1, 2}; when(client.create(any(BlobInfo.class), any(byte[].class))).thenReturn(blob); - Blob actual = new ArtifactGcsSdkClient(client).uploadArtifact(BUCKET, DIR_PATH, LOCAL_PATH); + GcsArtifact actual = (GcsArtifact) artifactClient.createArtifact(artifactName, contents); verify(client).create(blobInfoCaptor.capture(), contentsCaptor.capture()); + BlobInfo actualInfo = blobInfoCaptor.getValue(); + + assertThat(actual.blob).isSameInstanceAs(blob); + assertThat(actualInfo.getBucket()).isEqualTo(BUCKET); + assertThat(actualInfo.getName()) + .isEqualTo(String.format("%s/%s/%s", TEST_CLASS, artifactClient.runId(), artifactName)); + assertThat(contentsCaptor.getValue()).isEqualTo(contents); + } + + @Test + public void testUploadArtifact() throws IOException { + when(client.create(any(BlobInfo.class), any(byte[].class))).thenReturn(blob); + GcsArtifact actual = (GcsArtifact) artifactClient.uploadArtifact(ARTIFACT_NAME, LOCAL_PATH); + + verify(client).create(blobInfoCaptor.capture(), contentsCaptor.capture()); BlobInfo actualInfo = blobInfoCaptor.getValue(); - byte[] actualContents = contentsCaptor.getValue(); - assertThat(actual).isSameInstanceAs(blob); + assertThat(actual.blob).isSameInstanceAs(blob); assertThat(actualInfo.getBucket()).isEqualTo(BUCKET); - assertThat(actualInfo.getName()).isEqualTo(DIR_PATH); - assertThat(actualContents).isEqualTo(TEST_ARTIFACT_CONTENTS); + assertThat(actualInfo.getName()) + .isEqualTo(String.format("%s/%s/%s", TEST_CLASS, artifactClient.runId(), ARTIFACT_NAME)); + assertThat(contentsCaptor.getValue()).isEqualTo(TEST_ARTIFACT_CONTENTS); } @Test @@ -101,18 +142,15 @@ public void testUploadArtifactInvalidLocalPath() { when(client.create(any(BlobInfo.class), any())).thenReturn(blob); assertThrows( IOException.class, - () -> - new ArtifactGcsSdkClient(client) - .uploadArtifact( - BUCKET, DIR_PATH, "/some/invalid/path/please/do/not/make/this/file/valid")); + () -> artifactClient.uploadArtifact(ARTIFACT_NAME, "/" + UUID.randomUUID())); } @Test - public void testListArtifactsSinglePage() { + public void testListArtifactsInMethodDirSinglePage() { // Arrange - String name1 = DIR_PATH + "/blob1"; - String name2 = DIR_PATH + "/blob2"; - String name3 = DIR_PATH + "/blob3"; + String name1 = "blob1"; + String name2 = "blob2"; + String name3 = "blob3"; ImmutableList page1 = ImmutableList.of(mock(Blob.class), mock(Blob.class), mock(Blob.class)); when(page1.get(0).getName()).thenReturn(name1); @@ -125,7 +163,7 @@ public void testListArtifactsSinglePage() { Pattern pattern = Pattern.compile(".*blob[13].*"); // Act - List actual = new ArtifactGcsSdkClient(client).listArtifacts(BUCKET, DIR_PATH, pattern); + List actual = artifactClient.listArtifacts(TEST_METHOD, pattern); // Assert verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); @@ -134,18 +172,21 @@ public void testListArtifactsSinglePage() { BlobListOption actualOptions = listOptionsCaptor.getValue(); assertThat(actual).hasSize(2); - assertThat(actual.get(0).getName()).isEqualTo(name1); - assertThat(actual.get(1).getName()).isEqualTo(name3); + assertThat(actual.get(0).name()).isEqualTo(name1); + assertThat(actual.get(1).name()).isEqualTo(name3); assertThat(actualBucket).isEqualTo(BUCKET); - assertThat(actualOptions).isEqualTo(BucketListOption.prefix(DIR_PATH)); + assertThat(actualOptions) + .isEqualTo( + BucketListOption.prefix( + String.format("%s/%s/%s", TEST_CLASS, artifactClient.runId(), TEST_METHOD))); } @Test - public void testListArtifactsMultiplePages() { + public void testListArtifactsInMethodDirMultiplePages() { // Arrange - String name1 = DIR_PATH + "/blob1"; - String name2 = DIR_PATH + "/blob2"; - String name3 = DIR_PATH + "/blob3"; + String name1 = "blob1"; + String name2 = "blob2"; + String name3 = "blob3"; ImmutableList page1 = ImmutableList.of(mock(Blob.class), mock(Blob.class)); ImmutableList page2 = ImmutableList.of(mock(Blob.class)); when(page1.get(0).getName()).thenReturn(name1); @@ -158,7 +199,7 @@ public void testListArtifactsMultiplePages() { Pattern pattern = Pattern.compile(".*blob[13].*"); // Act - List actual = new ArtifactGcsSdkClient(client).listArtifacts(BUCKET, DIR_PATH, pattern); + List actual = artifactClient.listArtifacts(TEST_METHOD, pattern); // Assert verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); @@ -167,30 +208,33 @@ public void testListArtifactsMultiplePages() { BlobListOption actualOptions = listOptionsCaptor.getValue(); assertThat(actual).hasSize(2); - assertThat(actual.get(0).getName()).isEqualTo(name1); - assertThat(actual.get(1).getName()).isEqualTo(name3); + assertThat(actual.get(0).name()).isEqualTo(name1); + assertThat(actual.get(1).name()).isEqualTo(name3); assertThat(actualBucket).isEqualTo(BUCKET); - assertThat(actualOptions).isEqualTo(BucketListOption.prefix(DIR_PATH)); + assertThat(actualOptions) + .isEqualTo( + BucketListOption.prefix( + String.format("%s/%s/%s", TEST_CLASS, artifactClient.runId(), TEST_METHOD))); } @Test - public void testListArtifactsNoArtifacts() { + public void testListArtifactsInMethodDirNoArtifacts() { TestBlobPage allPages = createPages(ImmutableList.of()); when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); Pattern pattern = Pattern.compile(".*blob[13].*"); - List actual = new ArtifactGcsSdkClient(client).listArtifacts(BUCKET, DIR_PATH, pattern); + List actual = artifactClient.listArtifacts(TEST_METHOD, pattern); verify(client).list(anyString(), any(BlobListOption.class)); assertThat(actual).isEmpty(); } @Test - public void testDeleteArtifactsSinglePage() { + public void testCleanupRunSinglePage() { // Arrange - BlobId id1 = BlobId.of(BUCKET, DIR_PATH + "/blob1"); - BlobId id2 = BlobId.of(BUCKET, DIR_PATH + "/blob2"); - BlobId id3 = BlobId.of(BUCKET, DIR_PATH + "/blob3"); + BlobId id1 = BlobId.of(BUCKET, "blob1"); + BlobId id2 = BlobId.of(BUCKET, "blob2"); + BlobId id3 = BlobId.of(BUCKET, "blob3"); ImmutableList page1 = ImmutableList.of(mock(Blob.class), mock(Blob.class), mock(Blob.class)); when(page1.get(0).getBlobId()).thenReturn(id1); @@ -203,7 +247,7 @@ public void testDeleteArtifactsSinglePage() { when(client.delete(anyIterable())).thenReturn(ImmutableList.of(true, false, true)); // Act - new ArtifactGcsSdkClient(client).deleteTestDir(BUCKET, DIR_PATH); + artifactClient.cleanupRun(); // Assert verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); @@ -214,16 +258,18 @@ public void testDeleteArtifactsSinglePage() { Iterable actualIds = blobIdCaptor.getValue(); assertThat(actualBucket).isEqualTo(BUCKET); - assertThat(actualOption).isEqualTo(BucketListOption.prefix(DIR_PATH)); + assertThat(actualOption) + .isEqualTo( + BucketListOption.prefix(String.format("%s/%s", TEST_CLASS, artifactClient.runId()))); assertThat(actualIds).containsExactly(id1, id2, id3); } @Test - public void testDeleteArtifactsMultiplePages() { + public void testCleanupRunMultiplePages() { // Arrange - BlobId id1 = BlobId.of(BUCKET, DIR_PATH + "/blob1"); - BlobId id2 = BlobId.of(BUCKET, DIR_PATH + "/blob2"); - BlobId id3 = BlobId.of(BUCKET, DIR_PATH + "/blob3"); + BlobId id1 = BlobId.of(BUCKET, "blob1"); + BlobId id2 = BlobId.of(BUCKET, "blob2"); + BlobId id3 = BlobId.of(BUCKET, "blob3"); ImmutableList page1 = ImmutableList.of(mock(Blob.class), mock(Blob.class)); ImmutableList page2 = ImmutableList.of(mock(Blob.class)); when(page1.get(0).getBlobId()).thenReturn(id1); @@ -233,12 +279,12 @@ public void testDeleteArtifactsMultiplePages() { TestBlobPage allPages = createPages(page1, page2); when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); - // Technically, the second value is not realistic for the second call to delete, but it - // shouldn't mess anything up - when(client.delete(anyIterable())).thenReturn(ImmutableList.of(true, false)); + when(client.delete(anyIterable())) + .thenReturn(ImmutableList.of(true, false)) + .thenReturn(ImmutableList.of(true)); // Act - new ArtifactGcsSdkClient(client).deleteTestDir(BUCKET, DIR_PATH); + artifactClient.cleanupRun(); // Assert verify(client).list(bucketCaptor.capture(), listOptionsCaptor.capture()); @@ -249,7 +295,9 @@ public void testDeleteArtifactsMultiplePages() { List> actualBlobIds = blobIdCaptor.getAllValues(); assertThat(actualBucket).isEqualTo(BUCKET); - assertThat(actualOption).isEqualTo(BucketListOption.prefix(DIR_PATH)); + assertThat(actualOption) + .isEqualTo( + BucketListOption.prefix(String.format("%s/%s", TEST_CLASS, artifactClient.runId()))); assertThat(actualBlobIds.get(0)).containsExactly(id1, id2); assertThat(actualBlobIds.get(1)).containsExactly(id3); } @@ -259,7 +307,7 @@ public void testDeleteArtifactsNoArtifacts() { TestBlobPage allPages = createPages(ImmutableList.of()); when(client.list(anyString(), any(BlobListOption.class))).thenReturn(allPages); - new ArtifactGcsSdkClient(client).deleteTestDir(BUCKET, DIR_PATH); + artifactClient.cleanupRun(); verify(client, never()).delete(anyIterable()); } diff --git a/it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactTest.java b/it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactTest.java new file mode 100644 index 0000000000..627b33d897 --- /dev/null +++ b/it/src/test/java/com/google/cloud/teleport/it/artifacts/GcsArtifactTest.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2022 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.it.artifacts; + +import static com.google.common.truth.Truth.assertThat; +import static org.mockito.Mockito.when; + +import com.google.cloud.storage.Blob; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +/** Unit tests for {@link GcsArtifact}. */ +@RunWith(JUnit4.class) +public class GcsArtifactTest { + @Rule public final MockitoRule mockito = MockitoJUnit.rule(); + + @Mock private Blob blob; + private GcsArtifact artifact; + + @Before + public void setUp() { + artifact = new GcsArtifact(blob); + } + + @Test + public void testId() { + String id = "test-id"; + when(blob.getGeneratedId()).thenReturn(id); + assertThat(artifact.id()).isEqualTo(id); + } + + @Test + public void testName() { + String name = "test-name"; + when(blob.getName()).thenReturn(name); + assertThat(artifact.name()).isEqualTo(name); + } + + @Test + public void testContents() { + byte[] contents = new byte[] {0, 1, 2}; + when(blob.getContent()).thenReturn(contents); + assertThat(artifact.contents()).isEqualTo(contents); + } +} diff --git a/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java index 990ff25c20..cbcc2ead1f 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/AbstractDataflowTemplateClientTest.java @@ -123,7 +123,7 @@ private static final class FakeDataflowTemplateClient extends AbstractDataflowTe } @Override - public JobInfo launchTemplate(String project, String region, LaunchOptions options) + public JobInfo launchTemplate(String project, String region, LaunchConfig options) throws IOException { throw new UnsupportedOperationException(); } diff --git a/it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java index c3c93d7700..d5655681f9 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/FlexTemplateClientTest.java @@ -34,7 +34,7 @@ import com.google.auth.Credentials; import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobInfo; import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; -import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.LaunchOptions; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.LaunchConfig; import com.google.common.collect.ImmutableMap; import java.io.IOException; import org.junit.Rule; @@ -86,8 +86,8 @@ public void testLaunchNewJob() throws IOException { Job getJob = new Job().setId(JOB_ID).setCurrentState(JobState.QUEUED.toString()); LaunchFlexTemplateResponse response = new LaunchFlexTemplateResponse().setJob(launchJob); - LaunchOptions options = - LaunchOptions.builder(JOB_NAME, SPEC_PATH).addParameter(PARAM_KEY, PARAM_VALUE).build(); + LaunchConfig options = + LaunchConfig.builder(JOB_NAME, SPEC_PATH).addParameter(PARAM_KEY, PARAM_VALUE).build(); when(getFlexTemplates(client).launch(any(), any(), any())).thenReturn(launch); when(getLocationJobs(client).get(any(), any(), any())).thenReturn(get); @@ -130,7 +130,7 @@ public void testLaunchNewJobThrowsException() throws IOException { () -> FlexTemplateClient.withDataflowClient(client) .launchTemplate( - PROJECT, REGION, LaunchOptions.builder(JOB_NAME, SPEC_PATH).build())); + PROJECT, REGION, LaunchConfig.builder(JOB_NAME, SPEC_PATH).build())); } private static Locations.Jobs getLocationJobs(Dataflow client) { diff --git a/v2/pom.xml b/v2/pom.xml index 9fd5502901..105a52ede7 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -151,7 +151,7 @@ com.google.cloud.teleport - it + integration-testing-lib 1.0-SNAPSHOT test diff --git a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java index 8aea312424..b77274880f 100644 --- a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java +++ b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java @@ -16,22 +16,22 @@ package com.google.cloud.teleport.v2.templates; import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createGcsClient; -import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createTestPath; -import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.createTestSuiteDirPath; +import static com.google.cloud.teleport.it.artifacts.ArtifactUtils.getFullGcsPath; import static com.google.cloud.teleport.it.dataflow.DataflowUtils.createJobName; import static com.google.common.truth.Truth.assertThat; -import com.google.cloud.storage.Blob; +import com.google.auth.Credentials; import com.google.cloud.storage.Storage; import com.google.cloud.teleport.it.TestProperties; +import com.google.cloud.teleport.it.artifacts.Artifact; import com.google.cloud.teleport.it.artifacts.ArtifactClient; -import com.google.cloud.teleport.it.artifacts.ArtifactGcsSdkClient; +import com.google.cloud.teleport.it.artifacts.GcsArtifactClient; import com.google.cloud.teleport.it.dataflow.DataflowOperator; import com.google.cloud.teleport.it.dataflow.DataflowOperator.Result; import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient; import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobInfo; import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.JobState; -import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.LaunchOptions; +import com.google.cloud.teleport.it.dataflow.DataflowTemplateClient.LaunchConfig; import com.google.cloud.teleport.it.dataflow.FlexTemplateClient; import com.google.cloud.teleport.v2.templates.StreamingDataGenerator.SinkType; import com.google.common.io.Resources; @@ -47,14 +47,16 @@ /** Integration test for {@link StreamingDataGenerator}. */ @RunWith(JUnit4.class) public final class StreamingDataGeneratorIT { - private static final TestProperties PROPERTIES = new TestProperties(); + private static final String ARTIFACT_BUCKET = TestProperties.artifactBucket(); + private static final Credentials CREDENTIALS = TestProperties.googleCredentials(); + private static final String PROJECT = TestProperties.project(); + private static final String REGION = TestProperties.region(); + private static final String SPEC_PATH = TestProperties.specPath(); private static final String SCHEMA_FILE = "gameevent.json"; private static final String LOCAL_SCHEMA_PATH = Resources.getResource(SCHEMA_FILE).getPath(); private static final String TEST_ROOT_DIR = "streaming-data-generator"; - private static final String TEST_DIR = createTestSuiteDirPath(TEST_ROOT_DIR); - private static final String SCHEMA_FILE_GCS_PATH = String.format("%s/%s", TEST_DIR, SCHEMA_FILE); private static final String NUM_SHARDS_KEY = "numShards"; private static final String OUTPUT_DIRECTORY_KEY = "outputDirectory"; @@ -70,39 +72,38 @@ public final class StreamingDataGeneratorIT { @BeforeClass public static void setUpClass() throws IOException { - Storage gcsClient = createGcsClient(PROPERTIES.googleCredentials()); - artifactClient = new ArtifactGcsSdkClient(gcsClient); - artifactClient.uploadArtifact( - PROPERTIES.artifactBucket(), SCHEMA_FILE_GCS_PATH, LOCAL_SCHEMA_PATH); + Storage gcsClient = createGcsClient(CREDENTIALS); + artifactClient = GcsArtifactClient.builder(gcsClient, ARTIFACT_BUCKET, TEST_ROOT_DIR).build(); + artifactClient.uploadArtifact(SCHEMA_FILE, LOCAL_SCHEMA_PATH); } @AfterClass public static void tearDownClass() { - artifactClient.deleteTestDir(PROPERTIES.artifactBucket(), TEST_DIR); + artifactClient.cleanupRun(); } @Test public void testFakeMessagesToGcs() throws IOException { + // Arrange String name = "teleport-flex-streaming-data-generator-gcs"; - String outputDir = createTestPath(TEST_DIR, name); String jobName = createJobName(name); - LaunchOptions options = - LaunchOptions.builder(jobName, PROPERTIES.specPath()) - .addParameter( - SCHEMA_LOCATION_KEY, - String.format("gs://%s/%s", PROPERTIES.artifactBucket(), SCHEMA_FILE_GCS_PATH)) + + LaunchConfig options = + LaunchConfig.builder(jobName, SPEC_PATH) + // TODO(zhoufek): See if it is possible to use the properties interface and generate + // the map from the set values. + .addParameter(SCHEMA_LOCATION_KEY, getGcsSchemaLocation(SCHEMA_FILE)) .addParameter(QPS_KEY, DEFAULT_QPS) .addParameter(SINK_TYPE_KEY, SinkType.GCS.name()) .addParameter(WINDOW_DURATION_KEY, DEFAULT_WINDOW_DURATION) - .addParameter( - OUTPUT_DIRECTORY_KEY, - String.format("gs://%s/%s", PROPERTIES.artifactBucket(), outputDir)) + .addParameter(OUTPUT_DIRECTORY_KEY, getTestMethodDirPath(name)) .addParameter(NUM_SHARDS_KEY, "1") .build(); DataflowTemplateClient dataflow = - FlexTemplateClient.builder().setCredentials(PROPERTIES.googleCredentials()).build(); + FlexTemplateClient.builder().setCredentials(CREDENTIALS).build(); - JobInfo info = dataflow.launchTemplate(PROPERTIES.project(), PROPERTIES.region(), options); + // Act + JobInfo info = dataflow.launchTemplate(PROJECT, REGION, options); assertThat(info.state()).isIn(JobState.RUNNING_STATES); Result result = @@ -110,19 +111,28 @@ public void testFakeMessagesToGcs() throws IOException { .waitForConditionAndFinish( createConfig(info), () -> { - List outputFiles = - artifactClient.listArtifacts( - PROPERTIES.artifactBucket(), outputDir, Pattern.compile(".*output-.*")); + List outputFiles = + artifactClient.listArtifacts(name, Pattern.compile(".*output-.*")); return !outputFiles.isEmpty(); }); + + // Assert assertThat(result).isEqualTo(Result.CONDITION_MET); } + private static String getTestMethodDirPath(String testMethod) { + return getFullGcsPath(ARTIFACT_BUCKET, TEST_ROOT_DIR, artifactClient.runId(), testMethod); + } + + private static String getGcsSchemaLocation(String schemaFile) { + return getFullGcsPath(ARTIFACT_BUCKET, TEST_ROOT_DIR, artifactClient.runId(), schemaFile); + } + private static DataflowOperator.Config createConfig(JobInfo info) { return DataflowOperator.Config.builder() .setJobId(info.jobId()) - .setProject(PROPERTIES.project()) - .setRegion(PROPERTIES.region()) + .setProject(PROJECT) + .setRegion(REGION) .build(); } } From 1a1d3b0424aa7412ace3657df3728407b6eaabdb Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 2 Feb 2022 10:47:18 -0500 Subject: [PATCH 073/145] Add instructions for running an integration test --- it/README.md | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/it/README.md b/it/README.md index 944265728d..9d8c0cd017 100644 --- a/it/README.md +++ b/it/README.md @@ -9,3 +9,34 @@ or Flex Templates. The integration tests themselves. Those should go in the test folder for the template that they are being written for. + +## Running A Test + +NOTE: All commands are run from the root of the repository. + +Build this library: + +```shell +mvn clean install -f it/pom.xml +``` + +Authorize yourself in Google Cloud and set the required access token variable: + +```shell +gcloud auth application-default login +export DT_IT_ACCESS_TOKEN=$(gcloud auth application-default print-access-token) +``` + +Run the test (using a Flex Template): + +```shell +mvn clean test -f v2/pom.xml -pl "$MODULE" \ + -Dtest="$TEST_CLASS#$TEST_METHOD" \ + -Dproject="$PROJECT" \ + -DartifactBucket="$BUCKET" \ + -Dregion="$REGION" \ + -DspecPath="$SPEC_PATH" +``` + +NOTE: All the above args are required except `-Dregion`. If it is not provided, +then the template will run in `us-central1`. From ce9eb43d7aacf7a109d191f8010fecc72a3f141b Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Tue, 8 Mar 2022 11:46:14 -0800 Subject: [PATCH 074/145] Oracle2bq converts from date to date. PiperOrigin-RevId: 433269661 --- .../com/google/cloud/teleport/v2/utils/DataStreamClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java index 1bfa63fe64..5bb566a288 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java @@ -386,7 +386,7 @@ public StandardSQLTypeName convertOracleToBigQueryColumnType(OracleColumn column case "LONG_RAW": return StandardSQLTypeName.BYTES; case "DATE": - return StandardSQLTypeName.TIMESTAMP; + return StandardSQLTypeName.DATE; default: } From 48d521ead9425d28732ad3954060c440e4b40c56 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Tue, 8 Mar 2022 14:09:38 -0800 Subject: [PATCH 075/145] Add support for Secret Manager in Pub/Sub to Splunk Template. PiperOrigin-RevId: 433306801 --- pom.xml | 17 +- .../teleport/templates/PubSubToSplunk.java | 22 +- .../templates/common/SplunkConverters.java | 12 + .../templates/common/SplunkTokenSource.java | 29 +++ .../util/SecretManagerValueProvider.java | 94 ++++++++ .../util/TokenNestedValueProvider.java | 177 ++++++++++++++ .../util/TokenNestedValueProviderTest.java | 224 ++++++++++++++++++ 7 files changed, 559 insertions(+), 16 deletions(-) create mode 100644 src/main/java/com/google/cloud/teleport/templates/common/SplunkTokenSource.java create mode 100644 src/main/java/com/google/cloud/teleport/util/SecretManagerValueProvider.java create mode 100644 src/main/java/com/google/cloud/teleport/util/TokenNestedValueProvider.java create mode 100644 src/test/java/com/google/cloud/teleport/util/TokenNestedValueProviderTest.java diff --git a/pom.xml b/pom.xml index 1f4cd1cc33..546274597c 100644 --- a/pom.xml +++ b/pom.xml @@ -89,6 +89,7 @@ 1.40.0 0.87.0 2.12.1 + 24.2.0 @@ -100,8 +101,15 @@ pom import + + com.google.cloud + libraries-bom + ${cloud-libraries-bom.version} + pom + import + - + @@ -478,6 +486,10 @@ proto ${tensorflow.version} + + com.google.cloud + google-cloud-secretmanager + @@ -661,6 +673,9 @@ org.apache.beam:beam-vendor-grpc-1_43_2 + + com.google.api.grpc:proto-google-cloud-secretmanager-v1 + org.threeten:threetenbp diff --git a/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java b/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java index 43b6a221c9..788b2800e0 100644 --- a/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java +++ b/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java @@ -27,7 +27,7 @@ import com.google.cloud.teleport.templates.common.PubsubConverters.PubsubWriteDeadletterTopicOptions; import com.google.cloud.teleport.templates.common.SplunkConverters; import com.google.cloud.teleport.templates.common.SplunkConverters.SplunkOptions; -import com.google.cloud.teleport.util.KMSEncryptedNestedValueProvider; +import com.google.cloud.teleport.util.TokenNestedValueProvider; import com.google.cloud.teleport.values.FailsafeElement; import com.google.common.annotations.VisibleForTesting; import com.google.gson.Gson; @@ -245,7 +245,12 @@ public static PipelineResult run(PubSubToSplunkOptions options) { .apply( "WriteToSplunk", SplunkIO.writeBuilder() - .withToken(maybeDecrypt(options.getToken(), options.getTokenKMSEncryptionKey())) + .withToken( + new TokenNestedValueProvider( + options.getTokenSecretId(), + options.getTokenKMSEncryptionKey(), + options.getToken(), + options.getTokenSource())) .withUrl(options.getUrl()) .withBatchCount(options.getBatchCount()) .withParallelism(options.getParallelism()) @@ -367,19 +372,6 @@ public void processElement(ProcessContext context) { } } - /** - * Utility method to decrypt a Splunk HEC token. - * - * @param unencryptedToken The Splunk HEC token as a Base64 encoded {@link String} encrypted with - * a Cloud KMS Key. - * @param kmsKey The Cloud KMS Encryption Key to decrypt the Splunk HEC token. - * @return Decrypted Splunk HEC token. - */ - private static ValueProvider maybeDecrypt( - ValueProvider unencryptedToken, ValueProvider kmsKey) { - return new KMSEncryptedNestedValueProvider(unencryptedToken, kmsKey); - } - /** * Utility method that formats {@link org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage} according * to the model defined in {@link com.google.pubsub.v1.PubsubMessage}. diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java index c921d09fbb..7bb44bd034 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java @@ -133,6 +133,18 @@ public interface SplunkOptions extends PipelineOptions { void setTokenKMSEncryptionKey(ValueProvider keyName); + @Description( + "Secret Manager Secret ID for the token. Should be in the format " + + "projects/{project}/secrets/{secret}/versions/{secret_version}") + ValueProvider getTokenSecretId(); + + void setTokenSecretId(ValueProvider secretId); + + @Description("Source of the token. One of PLAINTEXT, KMS or SECRET_MANAGER.") + ValueProvider getTokenSource(); + + void setTokenSource(ValueProvider tokenSource); + @Description("Path to root CA in GCS, ex: gs://mybucket/somepath/rootCA.crt") ValueProvider getRootCaCertificatePath(); diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SplunkTokenSource.java b/src/main/java/com/google/cloud/teleport/templates/common/SplunkTokenSource.java new file mode 100644 index 0000000000..ab337967bf --- /dev/null +++ b/src/main/java/com/google/cloud/teleport/templates/common/SplunkTokenSource.java @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2021 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.common; + +/** Supported sources for a Splunk Token. */ +public enum SplunkTokenSource { + /** Token is passed as plaintext. */ + PLAINTEXT, + + /** Token to be decrypted using KMS. */ + KMS, + + /** Token is stored in Secret Manager. */ + SECRET_MANAGER +} diff --git a/src/main/java/com/google/cloud/teleport/util/SecretManagerValueProvider.java b/src/main/java/com/google/cloud/teleport/util/SecretManagerValueProvider.java new file mode 100644 index 0000000000..69d55422aa --- /dev/null +++ b/src/main/java/com/google/cloud/teleport/util/SecretManagerValueProvider.java @@ -0,0 +1,94 @@ +/* + * Copyright (C) 2021 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.util; + +import com.google.cloud.secretmanager.v1.AccessSecretVersionResponse; +import com.google.cloud.secretmanager.v1.SecretManagerServiceClient; +import com.google.cloud.secretmanager.v1.SecretVersionName; +import java.io.IOException; +import java.io.Serializable; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.SerializableFunction; + +/** + * {@link SecretManagerValueProvider} class is a subclass of {@link ValueProvider} that takes in a + * {@link ValueProvider} of the form + * projects/{project}/secrets/{secret}/versions/{secret_version} and returns the secret value in + * Secret Manager. + */ +public class SecretManagerValueProvider implements ValueProvider, Serializable { + + private transient volatile String cachedValue; + private final SerializableFunction translator; + private final ValueProvider secretVersion; + + @Override + public String get() { + if (cachedValue == null) { + cachedValue = translator.apply(secretVersion.get()); + } + return cachedValue; + } + + @Override + public boolean isAccessible() { + return secretVersion.isAccessible(); + } + + private static class SecretTranslatorInput implements SerializableFunction { + + private SecretTranslatorInput() {} + + public static SecretTranslatorInput of() { + return new SecretTranslatorInput(); + } + + @Override + public String apply(String secretVersion) { + SecretVersionName secretVersionName = parseSecretVersion(secretVersion); + + try (SecretManagerServiceClient client = SecretManagerServiceClient.create()) { + AccessSecretVersionResponse response = client.accessSecretVersion(secretVersionName); + return response.getPayload().getData().toStringUtf8(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Parses a Secret Version and returns a {@link SecretVersionName}. + * + * @param secretVersion Secret Version of the form + * projects/{project}/secrets/{secret}/versions/{secret_version} + * @return {@link SecretVersionName} + */ + private SecretVersionName parseSecretVersion(String secretVersion) { + if (SecretVersionName.isParsableFrom(secretVersion)) { + return SecretVersionName.parse(secretVersion); + } else { + throw new IllegalArgumentException( + "Provided Secret must be in the form" + + " projects/{project}/secrets/{secret}/versions/{secret_version}"); + } + } + } + + public SecretManagerValueProvider(ValueProvider secretVersion) { + this.secretVersion = secretVersion; + this.translator = SecretTranslatorInput.of(); + } +} diff --git a/src/main/java/com/google/cloud/teleport/util/TokenNestedValueProvider.java b/src/main/java/com/google/cloud/teleport/util/TokenNestedValueProvider.java new file mode 100644 index 0000000000..bd1d7f934a --- /dev/null +++ b/src/main/java/com/google/cloud/teleport/util/TokenNestedValueProvider.java @@ -0,0 +1,177 @@ +/* + * Copyright (C) 2021 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.util; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.cloud.teleport.templates.common.SplunkTokenSource; +import com.google.common.annotations.VisibleForTesting; +import java.io.Serializable; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.EnumUtils; +import org.apache.beam.sdk.options.ValueProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Returns a token from a valid {@link SplunkTokenSource}. */ +public class TokenNestedValueProvider implements ValueProvider, Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(TokenNestedValueProvider.class); + + private transient volatile String cachedValue; + + private final ValueProvider secretId; + private final ValueProvider kmsEncryptionKey; + private final ValueProvider token; + private final ValueProvider tokenSource; + + @Override + public String get() { + + if (cachedValue == null) { + + SplunkTokenSource finalTokenSource; + if (tokenSource.get() == null) { + // Since token source was introduced after KMS and plaintext options it may not be present. + // In that case we attempt to determine the token source based on whether only the plaintext + // token is present or both encrypted token and KMS Key params are present. + // Passing a tokenSource is mandatory if the token is stored in Secret Manager. + finalTokenSource = resolveTokenSource(); + } else { + finalTokenSource = EnumUtils.getEnum(SplunkTokenSource.class, tokenSource.get()); + checkArgument( + finalTokenSource != null, + "tokenSource must be one of PLAINTEXT, KMS or SECRET_MANAGER, but found: " + + tokenSource); + } + + cachedValue = getToken(finalTokenSource).get(); + } + + return cachedValue; + } + + @Override + public boolean isAccessible() { + return secretId.isAccessible() + || (kmsEncryptionKey.isAccessible() && token.isAccessible()) + || token.isAccessible(); + } + + public TokenNestedValueProvider( + ValueProvider secretId, + ValueProvider kmsEncryptionKey, + ValueProvider token, + ValueProvider tokenSource) { + this.secretId = secretId; + this.kmsEncryptionKey = kmsEncryptionKey; + this.token = token; + this.tokenSource = tokenSource; + } + + /** + * Utility method that attempts to determine if token source is KMS or PLAINTEXT based on user + * provided parameters. Added for backwards compatibility with the KMS and PLAINTEXT token + * options. + * + * @return {@link SplunkTokenSource} + */ + @VisibleForTesting + protected SplunkTokenSource resolveTokenSource() { + + if (tokenKmsParamsExist()) { + LOG.info("tokenSource set to {}", SplunkTokenSource.KMS); + return SplunkTokenSource.KMS; + + } else if (tokenPlaintextParamsExist()) { + LOG.info("tokenSource set to {}", SplunkTokenSource.PLAINTEXT); + return SplunkTokenSource.PLAINTEXT; + + } else { + throw new RuntimeException( + "Could not resolve tokenSource from given parameters. Pass in a tokenSource parameter" + + " with value one of SECRET_MANAGER, KMS or PLAINTEXT."); + } + } + + /** + * Helper method that checks if the Secret ID for required for retrieving the token from Secret + * Manager exists. + * + * @return true if Secret ID exists or false otherwise + */ + private boolean tokenSecretManagerParamsExist() { + return secretId.get() != null; + } + + /** + * Helper method that checks params required for decrypting the token using a KMS Key. + * + * @return true if encrypted token and KMS Key exist or false otherwise + */ + private boolean tokenKmsParamsExist() { + return token.get() != null && kmsEncryptionKey.get() != null; + } + + /** + * Helper method that checks if the plaintext token exists. + * + * @return true if plaintext token exists or false otherwise + */ + private boolean tokenPlaintextParamsExist() { + return token.get() != null; + } + + /** + * Utility method that retrieves the token from a valid {@link SplunkTokenSource}. + * + * @param tokenSource {@link SplunkTokenSource} + * @return {@link ValueProvider} Splunk HEC token + */ + @VisibleForTesting + protected ValueProvider getToken(SplunkTokenSource tokenSource) { + switch (tokenSource) { + case SECRET_MANAGER: + checkArgument( + tokenSecretManagerParamsExist(), + "tokenSecretId is required to retrieve token from Secret Manager"); + + LOG.info("Using token secret stored in Secret Manager"); + return new SecretManagerValueProvider(secretId); + + case KMS: + checkArgument( + tokenKmsParamsExist(), + "token and tokenKmsEncryptionKey are required while decrypting using KMS Key"); + + LOG.info("Using KMS Key to decrypt token"); + return new KMSEncryptedNestedValueProvider(token, kmsEncryptionKey); + + case PLAINTEXT: + checkArgument(tokenPlaintextParamsExist(), "token is required for writing events"); + + LOG.warn( + "Using plaintext token. Consider storing the token in Secret Manager or " + + "pass an encrypted token and a KMS Key to decrypt it"); + return token; + + default: + throw new RuntimeException( + "tokenSource must be one of PLAINTEXT, KMS or SECRET_MANAGER, but found: " + + tokenSource); + } + } +} diff --git a/src/test/java/com/google/cloud/teleport/util/TokenNestedValueProviderTest.java b/src/test/java/com/google/cloud/teleport/util/TokenNestedValueProviderTest.java new file mode 100644 index 0000000000..dddbb0dd30 --- /dev/null +++ b/src/test/java/com/google/cloud/teleport/util/TokenNestedValueProviderTest.java @@ -0,0 +1,224 @@ +/* + * Copyright (C) 2021 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.util; + +import static junit.framework.Assert.assertEquals; +import static junit.framework.Assert.assertTrue; + +import com.google.cloud.teleport.templates.common.SplunkTokenSource; +import org.apache.beam.sdk.options.ValueProvider; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link TokenNestedValueProvider}. */ +@RunWith(JUnit4.class) +public class TokenNestedValueProviderTest { + + @Rule public ExpectedException expectedException = ExpectedException.none(); + + /** + * Test that {@link TokenNestedValueProvider#resolveTokenSource()} correctly resolves the token + * source as {@code SplunkTokenSource.KMS} when a token and KMS Key are provided. + */ + @Test + public void testResolveTokenSource_kms() { + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of( + "projects/test-project/locations/test-region/keyRings/test-key-ring/cryptoKeys/test-key"), + ValueProvider.StaticValueProvider.of("test-token"), + ValueProvider.StaticValueProvider.of(null)); + + SplunkTokenSource tokenSource = tokenProvider.resolveTokenSource(); + + assertEquals(SplunkTokenSource.KMS, tokenSource); + } + + /** + * Test that {@link TokenNestedValueProvider#resolveTokenSource()} correctly resolves the token + * source as {@code SplunkTokenSource.PLAINTEXT} when only a plaintext token is provided. + */ + @Test + public void testResolveTokenSource_plaintext() { + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of("test-token"), + ValueProvider.StaticValueProvider.of(null)); + + SplunkTokenSource tokenSource = tokenProvider.resolveTokenSource(); + + assertEquals(SplunkTokenSource.PLAINTEXT, tokenSource); + } + + /** + * Test that {@link TokenNestedValueProvider#resolveTokenSource()} fails when a Secret Manager + * Secret ID is passed without a token source. + */ + @Test + public void testResolveTokenSource_secretManager() { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage( + "Could not resolve tokenSource from given parameters. Pass in a tokenSource parameter with" + + " value one of SECRET_MANAGER, KMS or PLAINTEXT."); + + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of( + "projects/test-project/secrets/test-secret/versions/test-version"), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null)); + + tokenProvider.resolveTokenSource(); + } + + /** + * Test that {@link TokenNestedValueProvider#getToken(SplunkTokenSource)} returns an instance of + * {@link SecretManagerValueProvider} when a Secret ID and 'SECRET_MANAGER' token source are + * provided. + */ + @Test + public void testGetToken_secretManager() { + ValueProvider tokenSource = ValueProvider.StaticValueProvider.of("SECRET_MANAGER"); + + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of( + "projects/test-project/secrets/test-secret/versions/test-version"), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + tokenSource); + + ValueProvider finalToken = + tokenProvider.getToken(SplunkTokenSource.valueOf(tokenSource.get())); + assertTrue(finalToken instanceof SecretManagerValueProvider); + } + + /** + * Test that {@link TokenNestedValueProvider#getToken(SplunkTokenSource)} fails when the token + * source is 'SECRET_MANAGER' and no Secret ID is provided. + */ + @Test + public void testGetToken_secretManagerInvalidParams() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "tokenSecretId is required to retrieve token from Secret Manager"); + + ValueProvider tokenSource = ValueProvider.StaticValueProvider.of("SECRET_MANAGER"); + + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + tokenSource); + + tokenProvider.getToken(SplunkTokenSource.valueOf(tokenSource.get())); + } + + /** + * Test that {@link TokenNestedValueProvider#getToken(SplunkTokenSource)} returns an instance of + * {@link KMSEncryptedNestedValueProvider} when a KMS Key, encrypted token and 'KMS' token source + * are provided. + */ + @Test + public void testGetToken_kms() { + ValueProvider tokenSource = ValueProvider.StaticValueProvider.of("KMS"); + + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of( + "projects/test-project/locations/test-region/keyRings/test-key-ring/cryptoKeys/test-key"), + ValueProvider.StaticValueProvider.of("test-token"), + tokenSource); + + ValueProvider finalToken = + tokenProvider.getToken(SplunkTokenSource.valueOf(tokenSource.get())); + assertTrue(finalToken instanceof KMSEncryptedNestedValueProvider); + } + + /** + * Test that {@link TokenNestedValueProvider#getToken(SplunkTokenSource)} fails when the token + * source is 'KMS' but no KMS Key or encrypted token params are provided. + */ + @Test + public void testGetToken_kmsInvalidParams() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage( + "token and tokenKmsEncryptionKey are required while decrypting using KMS Key"); + + ValueProvider tokenSource = ValueProvider.StaticValueProvider.of("KMS"); + + TokenNestedValueProvider token = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + tokenSource); + + token.getToken(SplunkTokenSource.valueOf(tokenSource.get())); + } + + /** + * Test that {@link TokenNestedValueProvider#getToken(SplunkTokenSource)} returns a plaintext + * token when the token source is 'PLAINTEXT'. + */ + @Test + public void testGetToken_plaintext() { + ValueProvider tokenSource = ValueProvider.StaticValueProvider.of("PLAINTEXT"); + ValueProvider expectedToken = ValueProvider.StaticValueProvider.of("test-token"); + + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + expectedToken, + tokenSource); + + ValueProvider actualToken = + tokenProvider.getToken(SplunkTokenSource.valueOf(tokenSource.get())); + assertEquals(expectedToken.get(), actualToken.get()); + } + + /** + * Test that {@link TokenNestedValueProvider#getToken(SplunkTokenSource)} fails when the token + * source is 'PLAINTEXT' but no plaintext token is provided. + */ + @Test + public void testGetToken_plaintextInvalidParams() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("token is required for writing events"); + + ValueProvider tokenSource = ValueProvider.StaticValueProvider.of("PLAINTEXT"); + + TokenNestedValueProvider tokenProvider = + new TokenNestedValueProvider( + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + ValueProvider.StaticValueProvider.of(null), + tokenSource); + + tokenProvider.getToken(SplunkTokenSource.valueOf(tokenSource.get())); + } +} From c423c8c6727becac61905a7147b23a696c4ad669 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 22 Feb 2022 12:18:54 -0500 Subject: [PATCH 076/145] Make it even easier to write tests and understand the new library --- cicd/internal/repo/modules_test.go | 4 ++-- example-input.txt | 2 -- it/pom.xml | 4 +++- .../cloud/teleport/it/artifacts/Artifact.java | 5 +++++ .../it/dataflow/DataflowTemplateClient.java | 2 +- .../teleport/it/dataflow/DataflowUtils.java | 16 +++++++++++++++- .../teleport/it/dataflow/DataflowUtilsTest.java | 6 +++++- unified-templates.xml | 3 ++- .../v2/templates/StreamingDataGeneratorIT.java | 10 +++++++--- 9 files changed, 40 insertions(+), 12 deletions(-) delete mode 100644 example-input.txt diff --git a/cicd/internal/repo/modules_test.go b/cicd/internal/repo/modules_test.go index ad70e3c030..903b85e816 100644 --- a/cicd/internal/repo/modules_test.go +++ b/cicd/internal/repo/modules_test.go @@ -59,7 +59,7 @@ func TestGetModulesForPaths(t *testing.T) { expected: map[string][]string{ ClassicRoot: []string{}, ItRoot: []string{}, - FlexRoot: []string{"pubsub-binary-to-bigquery"}, + FlexRoot: []string{"pubsub-binary-to-bigquery", ""}, }, }, { @@ -79,7 +79,7 @@ func TestGetModulesForPaths(t *testing.T) { expected: map[string][]string{ ClassicRoot: make([]string, 0), ItRoot: make([]string, 0), - FlexRoot: make([]string, 0), + FlexRoot: []string{""}, }, }, { diff --git a/example-input.txt b/example-input.txt deleted file mode 100644 index cf77958798..0000000000 --- a/example-input.txt +++ /dev/null @@ -1,2 +0,0 @@ -hello1 -hello2 diff --git a/it/pom.xml b/it/pom.xml index cd32dede75..4d186fba66 100644 --- a/it/pom.xml +++ b/it/pom.xml @@ -25,8 +25,9 @@ 1.8.2 2.34.0 24.2.0 + 2.2.2 1.8 - 4.13.2 + 4.13.1 31.0.1-jre 3.0.0 3.6.2 @@ -64,6 +65,7 @@ com.google.cloud google-cloud-storage + ${cloud-storage.version} diff --git a/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java b/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java index 45c3e8f238..525744ea97 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java +++ b/it/src/main/java/com/google/cloud/teleport/it/artifacts/Artifact.java @@ -18,6 +18,11 @@ /** * Represents a single artifact. * + *

An "artifact" is an entity in object storage, file storage, or block storage. Artifacts should + * be able to be stored in-memory as a single byte array. Implementations with an underlying type + * that only supports streaming should stream in the full contents and make the full contents + * available. + * *

Implementations should remain read-only. Writing artifacts should be left to the * responsibility of a {@link ArtifactClient} implementation. If an object of the artifact type * returned allows writing of any type, then it should not be made available. diff --git a/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java index 3b313dae24..3cf838edc8 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowTemplateClient.java @@ -44,7 +44,7 @@ enum JobState { private static final String DATAFLOW_PREFIX = "JOB_STATE_"; /** States that indicate the job is running or getting ready to run. */ - public static final ImmutableSet RUNNING_STATES = + public static final ImmutableSet ACTIVE_STATES = ImmutableSet.of(RUNNING, UPDATED, PENDING, QUEUED); /** States that indicate that the job is done. */ diff --git a/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java index 09d918d70e..5e976da374 100644 --- a/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java +++ b/it/src/main/java/com/google/cloud/teleport/it/dataflow/DataflowUtils.java @@ -26,6 +26,9 @@ private DataflowUtils() {} /** * Creates a job name. * + *

If there are uppercase characters in {@code prefix}, then this will convert them into a dash + * followed by the lowercase equivalent of that letter. + * *

The job name will normally be unique, but this is not guaranteed if multiple jobs with the * same prefix are requested in a short period of time. * @@ -34,9 +37,20 @@ private DataflowUtils() {} * prefix */ public static String createJobName(String prefix) { + StringBuilder properlyFormatted = new StringBuilder(); + for (int i = 0; i < prefix.length(); ++i) { + char c = prefix.charAt(i); + if (Character.isUpperCase(c)) { + properlyFormatted.append("-"); + properlyFormatted.append(Character.toLowerCase(c)); + } else { + properlyFormatted.append(c); + } + } + return String.format( "%s-%s", - prefix, + properlyFormatted, DateTimeFormatter.ofPattern("yyyyMMddHHmmss") .withZone(ZoneId.of("UTC")) .format(Instant.now())); diff --git a/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java index f02d6c6ba2..d41853975d 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowUtilsTest.java @@ -25,10 +25,14 @@ /** Unit tests for {@link DataflowUtils}. */ @RunWith(JUnit4.class) public class DataflowUtilsTest { - @Test public void testCreateJobName() { String name = "create-job-name"; assertThat(createJobName(name)).matches(name + "-\\d{14}"); } + + @Test + public void testCreateJobNameWithUppercase() { + assertThat(createJobName("testWithUpperCase")).matches("test-with-upper-case" + "-\\d{14}"); + } } diff --git a/unified-templates.xml b/unified-templates.xml index a2c36f848b..0e7bfd6058 100644 --- a/unified-templates.xml +++ b/unified-templates.xml @@ -54,6 +54,7 @@ . + it v2 - \ No newline at end of file + diff --git a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java index b77274880f..60705df23c 100644 --- a/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java +++ b/v2/streaming-data-generator/src/test/java/com/google/cloud/teleport/v2/templates/StreamingDataGeneratorIT.java @@ -40,13 +40,17 @@ import java.util.List; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TestName; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Integration test for {@link StreamingDataGenerator}. */ @RunWith(JUnit4.class) public final class StreamingDataGeneratorIT { + @Rule public final TestName testName = new TestName(); + private static final String ARTIFACT_BUCKET = TestProperties.artifactBucket(); private static final Credentials CREDENTIALS = TestProperties.googleCredentials(); private static final String PROJECT = TestProperties.project(); @@ -56,7 +60,7 @@ public final class StreamingDataGeneratorIT { private static final String SCHEMA_FILE = "gameevent.json"; private static final String LOCAL_SCHEMA_PATH = Resources.getResource(SCHEMA_FILE).getPath(); - private static final String TEST_ROOT_DIR = "streaming-data-generator"; + private static final String TEST_ROOT_DIR = StreamingDataGeneratorIT.class.getSimpleName(); private static final String NUM_SHARDS_KEY = "numShards"; private static final String OUTPUT_DIRECTORY_KEY = "outputDirectory"; @@ -85,7 +89,7 @@ public static void tearDownClass() { @Test public void testFakeMessagesToGcs() throws IOException { // Arrange - String name = "teleport-flex-streaming-data-generator-gcs"; + String name = testName.getMethodName(); String jobName = createJobName(name); LaunchConfig options = @@ -104,7 +108,7 @@ public void testFakeMessagesToGcs() throws IOException { // Act JobInfo info = dataflow.launchTemplate(PROJECT, REGION, options); - assertThat(info.state()).isIn(JobState.RUNNING_STATES); + assertThat(info.state()).isIn(JobState.ACTIVE_STATES); Result result = new DataflowOperator(dataflow) From fd64240e5081ee837df8b34eb8cb2818d1225c86 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 14 Mar 2022 04:21:37 -0700 Subject: [PATCH 077/145] Cleaning up and removing LocalSpannerIO PiperOrigin-RevId: 434427074 --- .../teleport/spanner/ApplyDDLTransform.java | 6 +- .../teleport/spanner/ExportTransform.java | 4 +- .../teleport/spanner/ImportTransform.java | 12 +- .../spanner/ReadInformationSchema.java | 6 +- .../teleport/spanner/TextImportTransform.java | 6 +- .../teleport/templates/SpannerToText.java | 4 +- .../templates/common/SpannerConverters.java | 10 +- .../gcp/spanner/ExposedSpannerAccessor.java | 59 - .../io/gcp/spanner/LocalBatchSpannerRead.java | 207 --- .../gcp/spanner/LocalCreateTransactionFn.java | 48 - .../gcp/spanner/LocalReadSpannerSchema.java | 123 -- .../sdk/io/gcp/spanner/LocalSpannerIO.java | 1561 ----------------- .../sdk/io/gcp/spanner/ReadOperation.java | 104 -- .../sdk/io/gcp/spanner/SpannerConfig.java | 189 -- .../beam/sdk/io/gcp/spanner/package-info.java | 18 - .../teleport/spanner/CompareDatabases.java | 6 +- .../teleport/templates/SpannerToTextTest.java | 4 +- 17 files changed, 29 insertions(+), 2338 deletions(-) delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerAccessor.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalCreateTransactionFn.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalReadSpannerSchema.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java delete mode 100644 src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java diff --git a/src/main/java/com/google/cloud/teleport/spanner/ApplyDDLTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ApplyDDLTransform.java index 1cb06c5b99..b60b27fea2 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ApplyDDLTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ApplyDDLTransform.java @@ -21,7 +21,7 @@ import com.google.spanner.admin.database.v1.UpdateDatabaseDdlMetadata; import java.util.List; import java.util.concurrent.ExecutionException; -import org.apache.beam.sdk.io.gcp.spanner.ExposedSpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.DoFn; @@ -63,11 +63,11 @@ public PCollection expand(PCollection input) { ParDo.of( new DoFn() { - private transient ExposedSpannerAccessor spannerAccessor; + private transient SpannerAccessor spannerAccessor; @Setup public void setup() { - spannerAccessor = ExposedSpannerAccessor.create(spannerConfig); + spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); } @Teardown diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java index 0f90f78172..9e9218dcfd 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java @@ -69,9 +69,9 @@ import org.apache.beam.sdk.io.WriteFilesResult; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.io.gcp.spanner.LocalSpannerIO; import org.apache.beam.sdk.io.gcp.spanner.ReadOperation; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Combine; @@ -345,7 +345,7 @@ public void processElement(ProcessContext c) { PCollection rows = tables.apply( "Read all rows from Spanner", - LocalSpannerIO.readAll().withTransaction(tx).withSpannerConfig(spannerConfig)); + SpannerIO.readAll().withTransaction(tx).withSpannerConfig(spannerConfig)); ValueProvider resource = ValueProvider.NestedValueProvider.of( diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java index c36f3759e5..d28383bbc0 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java @@ -55,9 +55,9 @@ import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.io.gcp.spanner.ExposedSpannerAccessor; -import org.apache.beam.sdk.io.gcp.spanner.LocalSpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; import org.apache.beam.sdk.io.gcp.spanner.SpannerWriteResult; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.ValueProvider; @@ -150,7 +150,7 @@ public void processElement(ProcessContext c) { schemas.apply("Build avro DDL", Combine.globally(AsList.fn())); PCollectionView tx = - begin.apply(LocalSpannerIO.createTransaction().withSpannerConfig(spannerConfig)); + begin.apply(SpannerIO.createTransaction().withSpannerConfig(spannerConfig)); PCollection informationSchemaDdl = begin.apply("Read Information Schema", new ReadInformationSchema(spannerConfig, tx)); @@ -245,7 +245,7 @@ public void processElement(ProcessContext c) { SpannerWriteResult result = mutations.apply( "Write mutations " + depth, - LocalSpannerIO.write() + SpannerIO.write() .withSchemaReadySignal(ddl) .withSpannerConfig(spannerConfig) .withCommitDeadline(Duration.standardMinutes(1)) @@ -348,7 +348,7 @@ private static class CreateTables extends PTransform { private final ValueProvider earlyIndexCreateFlag; private final ValueProvider ddlCreationTimeoutInMinutes; - private transient ExposedSpannerAccessor spannerAccessor; + private transient SpannerAccessor spannerAccessor; /* If the schema has a lot of DDL changes after dataload, its preferable to create * them before dataload. This provides the threshold for the early creation. @@ -397,7 +397,7 @@ public PCollectionTuple expand(PBegin begin) { @Setup public void setup() { - spannerAccessor = ExposedSpannerAccessor.create(spannerConfig); + spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); } @Teardown diff --git a/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java b/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java index 3ffb204419..4360d1c733 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java @@ -20,7 +20,7 @@ import com.google.cloud.spanner.BatchTransactionId; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.InformationSchemaScanner; -import org.apache.beam.sdk.io.gcp.spanner.ExposedSpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.transforms.Create; @@ -52,7 +52,7 @@ public PCollection expand(PBegin p) { private static class ReadInformationSchemaFn extends DoFn { private final SpannerConfig spannerConfig; - private transient ExposedSpannerAccessor spannerAccessor; + private transient SpannerAccessor spannerAccessor; private final PCollectionView tx; public ReadInformationSchemaFn(SpannerConfig spannerConfig, PCollectionView tx) { @@ -62,7 +62,7 @@ public ReadInformationSchemaFn(SpannerConfig spannerConfig, PCollectionView im @Override public PDone expand(PBegin begin) { PCollectionView tx = - begin.apply(LocalSpannerIO.createTransaction().withSpannerConfig(spannerConfig)); + begin.apply(SpannerIO.createTransaction().withSpannerConfig(spannerConfig)); PCollection ddl = begin.apply("Read Information Schema", new ReadInformationSchema(spannerConfig, tx)); @@ -184,7 +184,7 @@ public void processElement(ProcessContext c) { .apply("Wait for previous depth " + depth, Wait.on(previousComputation)) .apply( "Write mutations " + depth, - LocalSpannerIO.write() + SpannerIO.write() .withSpannerConfig(spannerConfig) .withCommitDeadline(Duration.standardMinutes(1)) .withMaxCumulativeBackoff(Duration.standardHours(2)) diff --git a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java index 8ab8cda275..bfafd303fe 100644 --- a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java +++ b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java @@ -26,9 +26,9 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.io.gcp.spanner.LocalSpannerIO; import org.apache.beam.sdk.io.gcp.spanner.ReadOperation; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; @@ -147,7 +147,7 @@ public static void main(String[] args) { // parameters at the pipeline execution time. .apply( "Read all records", - LocalSpannerIO.readAll().withTransaction(tx).withSpannerConfig(spannerConfig)) + SpannerIO.readAll().withTransaction(tx).withSpannerConfig(spannerConfig)) .apply( "Struct To Csv", MapElements.into(TypeDescriptors.strings()) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java index 050df2f493..99def36020 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java @@ -46,8 +46,8 @@ import java.util.function.BiFunction; import java.util.stream.Collectors; import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.io.gcp.spanner.ExposedSpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.ReadOperation; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.Default; @@ -174,7 +174,7 @@ public static Builder builder() { return new AutoValue_SpannerConverters_ExportTransform.Builder(); } - private ExposedSpannerAccessor spannerAccessor; + private SpannerAccessor spannerAccessor; private DatabaseClient databaseClient; // SpannerAccessor is not serialiazable, thus can't be passed as a mock so we need to pass @@ -183,7 +183,7 @@ public static Builder builder() { // TODO make SpannerAccessor serializable DatabaseClient getDatabaseClient(SpannerConfig spannerConfig) { if (databaseClient == null) { - this.spannerAccessor = ExposedSpannerAccessor.create(spannerConfig); + this.spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); return this.spannerAccessor.getDatabaseClient(); } else { return this.databaseClient; @@ -467,11 +467,11 @@ public CreateTransactionFnWithTimestamp( this.spannerSnapshotTime = spannerSnapshotTime; } - private transient ExposedSpannerAccessor spannerAccessor; + private transient SpannerAccessor spannerAccessor; @DoFn.Setup public void setup() throws Exception { - spannerAccessor = ExposedSpannerAccessor.create(config); + spannerAccessor = SpannerAccessor.getOrCreate(config); } @Teardown diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerAccessor.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerAccessor.java deleted file mode 100644 index 7a5c428df3..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerAccessor.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright (C) 2020 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 org.apache.beam.sdk.io.gcp.spanner; - -import com.google.cloud.spanner.BatchClient; -import com.google.cloud.spanner.DatabaseAdminClient; -import com.google.cloud.spanner.DatabaseClient; -import com.google.cloud.spanner.Spanner; - -/** - * Class is ported from Apache Beam and reexposed. Manages lifecycle of {@link DatabaseClient} and - * {@link Spanner} instances. - */ -public class ExposedSpannerAccessor implements AutoCloseable { - - public static ExposedSpannerAccessor create(SpannerConfig spannerConfig) { - SpannerAccessor accessor = SpannerAccessor.getOrCreate(spannerConfig); - return new ExposedSpannerAccessor(accessor); - } - - private final SpannerAccessor accessor; - - private ExposedSpannerAccessor(SpannerAccessor accessor) { - this.accessor = accessor; - } - - /** Returns Spanner client for Read/Write operations. */ - public DatabaseClient getDatabaseClient() { - return accessor.getDatabaseClient(); - } - - /** Returns Spanner client for batch operations. */ - public BatchClient getBatchClient() { - return accessor.getBatchClient(); - } - - /** Returns Spanner client for Database Administration. */ - public DatabaseAdminClient getDatabaseAdminClient() { - return accessor.getDatabaseAdminClient(); - } - - @Override - public void close() { - accessor.close(); - } -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java deleted file mode 100644 index c67b78a1c3..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalBatchSpannerRead.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Copyright (C) 2021 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 org.apache.beam.sdk.io.gcp.spanner; - -import com.google.auto.value.AutoValue; -import com.google.cloud.spanner.BatchReadOnlyTransaction; -import com.google.cloud.spanner.Options; -import com.google.cloud.spanner.Options.RpcPriority; -import com.google.cloud.spanner.Partition; -import com.google.cloud.spanner.ResultSet; -import com.google.cloud.spanner.Struct; -import com.google.cloud.spanner.TimestampBound; -import java.util.List; -import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Reshuffle; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; - -/** - * This transform reads from Cloud Spanner using the {@link com.google.cloud.spanner.BatchClient}. - * Reads from multiple partitions are executed concurrently yet in the same read-only transaction. - */ -@AutoValue -abstract class LocalBatchSpannerRead - extends PTransform, PCollection> { - - public static LocalBatchSpannerRead create( - SpannerConfig spannerConfig, - PCollectionView txView, - TimestampBound timestampBound) { - return new AutoValue_LocalBatchSpannerRead(spannerConfig, txView, timestampBound); - } - - abstract SpannerConfig getSpannerConfig(); - - @Nullable - abstract PCollectionView getTxView(); - - abstract TimestampBound getTimestampBound(); - - @Override - public PCollection expand(PCollection input) { - PCollectionView txView = getTxView(); - if (txView == null) { - Pipeline begin = input.getPipeline(); - LocalSpannerIO.CreateTransaction createTx = - LocalSpannerIO.createTransaction() - .withSpannerConfig(getSpannerConfig()) - .withTimestampBound(getTimestampBound()); - txView = begin.apply(createTx); - } - return input - .apply( - "Generate Partitions", - ParDo.of(new GeneratePartitionsFn(getSpannerConfig(), txView)).withSideInputs(txView)) - .apply("Shuffle partitions", Reshuffle.viaRandomKey()) - .apply( - "Read from Partitions", - ParDo.of(new ReadFromPartitionFn(getSpannerConfig(), txView)).withSideInputs(txView)); - } - - @VisibleForTesting - static class GeneratePartitionsFn extends DoFn { - - private final SpannerConfig config; - private final PCollectionView txView; - - private transient ExposedSpannerAccessor spannerAccessor; - - public GeneratePartitionsFn( - SpannerConfig config, PCollectionView txView) { - this.config = config; - this.txView = txView; - } - - @Setup - public void setup() throws Exception { - spannerAccessor = ExposedSpannerAccessor.create(config); - } - - @Teardown - public void teardown() throws Exception { - spannerAccessor.close(); - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - Transaction tx = c.sideInput(txView); - BatchReadOnlyTransaction context = - spannerAccessor.getBatchClient().batchReadOnlyTransaction(tx.transactionId()); - for (Partition p : execute(c.element(), context)) { - c.output(p); - } - } - - private List execute(ReadOperation op, BatchReadOnlyTransaction tx) { - if (config.getRpcPriority() != null && config.getRpcPriority().get() != null) { - return executeWithPriority(op, tx, config.getRpcPriority().get()); - } else { - return executeWithoutPriority(op, tx); - } - } - - private List executeWithoutPriority(ReadOperation op, BatchReadOnlyTransaction tx) { - // Query was selected. - if (op.getQuery() != null) { - return tx.partitionQuery(op.getPartitionOptions(), op.getQuery()); - } - // Read with index was selected. - if (op.getIndex() != null) { - return tx.partitionReadUsingIndex( - op.getPartitionOptions(), - op.getTable(), - op.getIndex(), - op.getKeySet(), - op.getColumns()); - } - // Read from table was selected. - return tx.partitionRead( - op.getPartitionOptions(), op.getTable(), op.getKeySet(), op.getColumns()); - } - - private List executeWithPriority( - ReadOperation op, BatchReadOnlyTransaction tx, RpcPriority rpcPriority) { - // Query was selected. - if (op.getQuery() != null) { - return tx.partitionQuery( - op.getPartitionOptions(), op.getQuery(), Options.priority(rpcPriority)); - } - // Read with index was selected. - if (op.getIndex() != null) { - return tx.partitionReadUsingIndex( - op.getPartitionOptions(), - op.getTable(), - op.getIndex(), - op.getKeySet(), - op.getColumns(), - Options.priority(rpcPriority)); - } - // Read from table was selected. - return tx.partitionRead( - op.getPartitionOptions(), - op.getTable(), - op.getKeySet(), - op.getColumns(), - Options.priority(rpcPriority)); - } - } - - private static class ReadFromPartitionFn extends DoFn { - - private final SpannerConfig config; - private final PCollectionView txView; - - private transient ExposedSpannerAccessor spannerAccessor; - - public ReadFromPartitionFn( - SpannerConfig config, PCollectionView txView) { - this.config = config; - this.txView = txView; - } - - @Setup - public void setup() throws Exception { - spannerAccessor = ExposedSpannerAccessor.create(config); - } - - @Teardown - public void teardown() throws Exception { - spannerAccessor.close(); - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - Transaction tx = c.sideInput(txView); - - BatchReadOnlyTransaction batchTx = - spannerAccessor.getBatchClient().batchReadOnlyTransaction(tx.transactionId()); - - Partition p = c.element(); - try (ResultSet resultSet = batchTx.execute(p)) { - while (resultSet.next()) { - Struct s = resultSet.getCurrentRowAsStruct(); - c.output(s); - } - } - } - } -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalCreateTransactionFn.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalCreateTransactionFn.java deleted file mode 100644 index 3e5f2a6ed0..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalCreateTransactionFn.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright (C) 2021 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 org.apache.beam.sdk.io.gcp.spanner; - -import com.google.cloud.spanner.BatchReadOnlyTransaction; -import org.apache.beam.sdk.transforms.DoFn; - -/** Creates a batch transaction. */ -class LocalCreateTransactionFn extends DoFn { - - private final LocalSpannerIO.CreateTransaction config; - - LocalCreateTransactionFn(LocalSpannerIO.CreateTransaction config) { - this.config = config; - } - - private transient ExposedSpannerAccessor spannerAccessor; - - @DoFn.Setup - public void setup() throws Exception { - spannerAccessor = ExposedSpannerAccessor.create(config.getSpannerConfig()); - } - - @Teardown - public void teardown() throws Exception { - spannerAccessor.close(); - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - BatchReadOnlyTransaction tx = - spannerAccessor.getBatchClient().batchReadOnlyTransaction(config.getTimestampBound()); - c.output(Transaction.create(tx.getBatchTransactionId())); - } -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalReadSpannerSchema.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalReadSpannerSchema.java deleted file mode 100644 index 18bf0d266f..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalReadSpannerSchema.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * Copyright (C) 2021 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 org.apache.beam.sdk.io.gcp.spanner; - -import com.google.cloud.spanner.DatabaseClient; -import com.google.cloud.spanner.ReadOnlyTransaction; -import com.google.cloud.spanner.ResultSet; -import com.google.cloud.spanner.Statement; -import org.apache.beam.sdk.transforms.DoFn; - -/** - * This {@link DoFn} reads Cloud Spanner 'information_schema.*' tables to build the {@link - * SpannerSchema}. - */ -class LocalReadSpannerSchema extends DoFn { - - private final SpannerConfig config; - - private transient ExposedSpannerAccessor spannerAccessor; - - public LocalReadSpannerSchema(SpannerConfig config) { - this.config = config; - } - - @Setup - public void setup() throws Exception { - spannerAccessor = ExposedSpannerAccessor.create(config); - } - - @Teardown - public void teardown() throws Exception { - spannerAccessor.close(); - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - SpannerSchema.Builder builder = SpannerSchema.builder(); - DatabaseClient databaseClient = spannerAccessor.getDatabaseClient(); - try (ReadOnlyTransaction tx = databaseClient.readOnlyTransaction()) { - ResultSet resultSet = readTableInfo(tx); - - while (resultSet.next()) { - String tableName = resultSet.getString(0); - String columnName = resultSet.getString(1); - String type = resultSet.getString(2); - - // TODO: This is a work-around to successfully build an SpannerSchema when type is - // NUMERIC/JSON. - // org.apache.beam.sdk.io.gcp.spanner.SpannerSchema$Column.parseSpannerType() does not - // support JSON. - if (type.startsWith("NUMERIC") || type.startsWith("JSON")) { - type = "STRING(MAX)"; - } - if (type.startsWith("ARRAY") || type.startsWith("ARRAY")) { - type = "ARRAY"; - } - - long cellsMutated = resultSet.getLong(3); - - builder.addColumn(tableName, columnName, type, cellsMutated); - } - - resultSet = readPrimaryKeyInfo(tx); - while (resultSet.next()) { - String tableName = resultSet.getString(0); - String columnName = resultSet.getString(1); - String ordering = resultSet.getString(2); - - builder.addKeyPart(tableName, columnName, "DESC".equalsIgnoreCase(ordering)); - } - } - c.output(builder.build()); - } - - private ResultSet readTableInfo(ReadOnlyTransaction tx) { - // retrieve schema information for all tables, as well as aggregating the - // number of indexes that cover each column. this will be used to estimate - // the number of cells (table column plus indexes) mutated in an upsert operation - // in order to stay below the 20k threshold - return tx.executeQuery( - Statement.of( - "SELECT" - + " c.table_name" - + " , c.column_name" - + " , c.spanner_type" - + " , (1 + COALESCE(t.indices, 0)) AS cells_mutated" - + " FROM (" - + " SELECT c.table_name, c.column_name, c.spanner_type, c.ordinal_position" - + " FROM information_schema.columns as c" - + " WHERE c.table_catalog = '' AND c.table_schema = '') AS c" - + " LEFT OUTER JOIN (" - + " SELECT t.table_name, t.column_name, COUNT(*) AS indices" - + " FROM information_schema.index_columns AS t " - + " WHERE t.index_name != 'PRIMARY_KEY' AND t.table_catalog = ''" - + " AND t.table_schema = ''" - + " GROUP BY t.table_name, t.column_name) AS t" - + " USING (table_name, column_name)" - + " ORDER BY c.table_name, c.ordinal_position")); - } - - private ResultSet readPrimaryKeyInfo(ReadOnlyTransaction tx) { - return tx.executeQuery( - Statement.of( - "SELECT t.table_name, t.column_name, t.column_ordering" - + " FROM information_schema.index_columns AS t " - + " WHERE t.index_name = 'PRIMARY_KEY' AND t.table_catalog = ''" - + " AND t.table_schema = ''" - + " ORDER BY t.table_name, t.ordinal_position")); - } -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java deleted file mode 100644 index 9c0e2c0539..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/LocalSpannerIO.java +++ /dev/null @@ -1,1561 +0,0 @@ -/* - * Copyright (C) 2021 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 org.apache.beam.sdk.io.gcp.spanner; - -import static org.apache.beam.sdk.io.gcp.spanner.MutationUtils.isPointDelete; -import static org.apache.beam.sdk.io.gcp.spanner.SpannerIO.WriteGrouped.decode; -import static org.apache.beam.sdk.io.gcp.spanner.SpannerIO.WriteGrouped.encode; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.auto.value.AutoValue; -import com.google.cloud.ServiceFactory; -import com.google.cloud.Timestamp; -import com.google.cloud.spanner.AbortedException; -import com.google.cloud.spanner.ErrorCode; -import com.google.cloud.spanner.KeySet; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Mutation.Op; -import com.google.cloud.spanner.Options; -import com.google.cloud.spanner.PartitionOptions; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerException; -import com.google.cloud.spanner.SpannerOptions; -import com.google.cloud.spanner.Statement; -import com.google.cloud.spanner.Struct; -import com.google.cloud.spanner.TimestampBound; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.OptionalInt; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.Distribution; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Reshuffle; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.transforms.Wait; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.Sleeper; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Stopwatch; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.UnsignedBytes; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Experimental {@link PTransform Transforms} for reading from and writing to Google Cloud Spanner. - * - *

Reading from Cloud Spanner

- * - *

To read from Cloud Spanner, apply {@link SpannerIO.Read} transformation. It will return a - * {@link PCollection} of {@link Struct Structs}, where each element represents an individual row - * returned from the read operation. Both Query and Read APIs are supported. See more information - * about reading from Cloud Spanner - * - *

To execute a query, specify a {@link SpannerIO.Read#withQuery(Statement)} or - * {@link SpannerIO.Read#withQuery(String)} during the construction of the transform. - * - *

{@code
- * PCollection rows = p.apply(
- *     SpannerIO.read()
- *         .withInstanceId(instanceId)
- *         .withDatabaseId(dbId)
- *         .withQuery("SELECT id, name, email FROM users"));
- * }
- * - *

To use the Read API, specify a {@link SpannerIO.Read#withTable(String) table name} and a - * {@link SpannerIO.Read#withColumns(List) list of columns}. - * - *

{@code
- * PCollection rows = p.apply(
- *    SpannerIO.read()
- *        .withInstanceId(instanceId)
- *        .withDatabaseId(dbId)
- *        .withTable("users")
- *        .withColumns("id", "name", "email"));
- * }
- * - *

To optimally read using index, specify the index name using {@link SpannerIO.Read#withIndex}. - * - *

The transform is guaranteed to be executed on a consistent snapshot of data, utilizing the - * power of read only transactions. Staleness of data can be controlled using {@link - * SpannerIO.Read#withTimestampBound} or {@link SpannerIO.Read#withTimestamp(Timestamp)} methods. Read more about transactions in - * Cloud Spanner. - * - *

It is possible to read several {@link PCollection PCollections} within a single transaction. - * Apply {@link SpannerIO#createTransaction()} transform, that lazily creates a transaction. The - * result of this transformation can be passed to read operation using {@link - * SpannerIO.Read#withTransaction(PCollectionView)}. - * - *

{@code
- * SpannerConfig spannerConfig = ...
- *
- * PCollectionView tx = p.apply(
- *    SpannerIO.createTransaction()
- *        .withSpannerConfig(spannerConfig)
- *        .withTimestampBound(TimestampBound.strong()));
- *
- * PCollection users = p.apply(
- *    SpannerIO.read()
- *        .withSpannerConfig(spannerConfig)
- *        .withQuery("SELECT name, email FROM users")
- *        .withTransaction(tx));
- *
- * PCollection tweets = p.apply(
- *    SpannerIO.read()
- *        .withSpannerConfig(spannerConfig)
- *        .withQuery("SELECT user, tweet, date FROM tweets")
- *        .withTransaction(tx));
- * }
- * - *

Writing to Cloud Spanner

- * - *

The Cloud Spanner {@link SpannerIO.Write} transform writes to Cloud Spanner by executing a - * collection of input row {@link Mutation Mutations}. The mutations are grouped into batches for - * efficiency. - * - *

To configure the write transform, create an instance using {@link #write()} and then specify - * the destination Cloud Spanner instance ({@link Write#withInstanceId(String)} and destination - * database ({@link Write#withDatabaseId(String)}). For example: - * - *

{@code
- * // Earlier in the pipeline, create a PCollection of Mutations to be written to Cloud Spanner.
- * PCollection mutations = ...;
- * // Write mutations.
- * SpannerWriteResult result = mutations.apply(
- *     "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database"));
- * }
- * - *

SpannerWriteResult

- * - * The {@link SpannerWriteResult SpannerWriteResult} object contains the results of the transform, - * including a {@link PCollection} of MutationGroups that failed to write, and a {@link PCollection} - * that can be used in batch pipelines as a completion signal to {@link - * org.apache.beam.sdk.transforms.Wait Wait.OnSignal} to indicate when all input has been written. - * Note that in streaming pipelines, this signal will never be triggered as the input is unbounded - * and this {@link PCollection} is using the {@link GlobalWindow}. - * - *

Batching

- * - *

To reduce the number of transactions sent to Spanner, the {@link Mutation Mutations} are - * grouped into batches The default maximum size of the batch is set to 1MB or 5000 mutated cells, - * or 500 rows (whichever is reached first). To override this use {@link - * Write#withBatchSizeBytes(long) withBatchSizeBytes()}, {@link Write#withMaxNumMutations(long) - * withMaxNumMutations()} or {@link Write#withMaxNumMutations(long) withMaxNumRows()}. Setting - * either to a small value or zero disables batching. - * - *

Note that the maximum - * size of a single transaction is 20,000 mutated cells - including cells in indexes. If you - * have a large number of indexes and are getting exceptions with message: INVALID_ARGUMENT: The - * transaction contains too many mutations you will need to specify a smaller number of {@code - * MaxNumMutations}. - * - *

The batches written are obtained from by grouping enough {@link Mutation Mutations} from the - * Bundle provided by Beam to form (by default) 1000 batches. This group of {@link Mutation - * Mutations} is then sorted by table and primary key, and the batches are created from the sorted - * group. Each batch will then have rows with keys that are 'close' to each other to optimise write - * performance. This grouping factor (number of batches) is controlled by the parameter {@link - * Write#withGroupingFactor(int) withGroupingFactor()}. - * - *

Note that each worker will need enough memory to hold {@code GroupingFactor x - * MaxBatchSizeBytes} Mutations, so if you have a large {@code MaxBatchSize} you may need to reduce - * {@code GroupingFactor} - * - *

Monitoring

- * - *

Several counters are provided for monitoring purpooses: - * - *

    - *
  • batchable_mutation_groups
    - * Counts the mutations that are batched for writing to Spanner. - *
  • unbatchable_mutation_groups
    - * Counts the mutations that can not be batched and are applied individually - either because - * they are too large to fit into a batch, or they are ranged deletes. - *
  • mutation_group_batches_received, mutation_group_batches_write_success, - * mutation_group_batches_write_failed
    - * Count the number of batches that are processed. If Failure Mode is set to {@link - * FailureMode#REPORT_FAILURES REPORT_FAILURES}, then failed batches will be split up and the - * individual mutation groups retried separately. - *
  • mutation_groups_received, mutation_groups_write_success, - * mutation_groups_write_fail
    - * Count the number of individual MutationGroups that are processed. - *
  • spanner_write_success, spanner_write_fail
    - * The number of writes to Spanner that have occurred. - *
  • spanner_write_retries
    - * The number of times a write is retried after a failure - either due to a timeout, or when - * batches fail and {@link FailureMode#REPORT_FAILURES REPORT_FAILURES} is set so that - * individual Mutation Groups are retried. - *
  • spanner_write_timeouts
    - * The number of timeouts that occur when writing to Spanner. Writes that timed out are - * retried after a backoff. Large numbers of timeouts suggest an overloaded Spanner instance. - *
  • spanner_write_total_latency_ms
    - * The total amount of time spent writing to Spanner, in milliseconds. - *
- * - *

Database Schema Preparation

- * - *

The Write transform reads the database schema on pipeline start to know which columns are used - * as primary keys of the tables and indexes. This is so that the transform knows how to sort the - * grouped Mutations by table name and primary key as described above. - * - *

If the database schema, any additional tables or indexes are created in the same pipeline then - * there will be a race condition, leading to a situation where the schema is read before the table - * is created its primary key will not be known. This will mean that the sorting/batching will not - * be optimal and performance will be reduced (warnings will be logged for rows using unknown - * tables) - * - *

To prevent this race condition, use {@link Write#withSchemaReadySignal(PCollection)} to pass a - * signal {@link PCollection} (for example the output of the transform that creates the table(s)) - * which will be used with {@link Wait.OnSignal} to prevent the schema from being read until it is - * ready. The Write transform will be paused until this signal {@link PCollection} is closed. - * - *

Transactions

- * - *

The transform does not provide same transactional guarantees as Cloud Spanner. In particular, - * - *

    - *
  • Individual Mutations are submitted atomically, but all Mutations are not submitted in the - * same transaction. - *
  • A Mutation is applied at least once; - *
  • If the pipeline was unexpectedly stopped, mutations that were already applied will not get - * rolled back. - *
- * - *

Use {@link MutationGroup MutationGroups} with the {@link WriteGrouped} transform to ensure - * that a small set mutations is bundled together. It is guaranteed that mutations in a {@link - * MutationGroup} are submitted in the same transaction. Note that a MutationGroup must not exceed - * the Spanner transaction limits. - * - *

{@code
- * // Earlier in the pipeline, create a PCollection of MutationGroups to be written to Cloud
- * Spanner.
- * PCollection mutationGroups = ...;
- * // Write mutation groups.
- * SpannerWriteResult result = mutationGroups.apply(
- *     "Write",
- *     SpannerIO.write().withInstanceId("instance").withDatabaseId("database").grouped());
- * }
- * - *

Streaming Support

- * - *

{@link SpannerIO.Write} can be used as a streaming sink, however as with batch mode note that - * the write order of individual {@link Mutation}/{@link MutationGroup} objects is not guaranteed. - */ -@Experimental(Kind.SOURCE_SINK) -public class LocalSpannerIO { - private static final Logger LOG = LoggerFactory.getLogger(SpannerIO.class); - - private static final long DEFAULT_BATCH_SIZE_BYTES = 1024L * 1024L; // 1 MB - // Max number of mutations to batch together. - private static final int DEFAULT_MAX_NUM_MUTATIONS = 5000; - // Max number of mutations to batch together. - private static final int DEFAULT_MAX_NUM_ROWS = 500; - // Multiple of mutation size to use to gather and sort mutations - private static final int DEFAULT_GROUPING_FACTOR = 1000; - - /** - * Creates an uninitialized instance of {@link Read}. Before use, the {@link Read} must be - * configured with a {@link Read#withInstanceId} and {@link Read#withDatabaseId} that identify the - * Cloud Spanner database. - */ - public static Read read() { - return new AutoValue_LocalSpannerIO_Read.Builder() - .setSpannerConfig(SpannerConfig.create()) - .setTimestampBound(TimestampBound.strong()) - .setReadOperation(ReadOperation.create()) - .setBatching(true) - .build(); - } - - /** - * A {@link PTransform} that works like {@link #read}, but executes read operations coming from a - * {@link PCollection}. - */ - public static ReadAll readAll() { - return new AutoValue_LocalSpannerIO_ReadAll.Builder() - .setSpannerConfig(SpannerConfig.create()) - .setTimestampBound(TimestampBound.strong()) - .setBatching(true) - .build(); - } - - /** - * Returns a transform that creates a batch transaction. By default, {@link - * TimestampBound#strong()} transaction is created, to override this use {@link - * CreateTransaction#withTimestampBound(TimestampBound)}. - */ - @Experimental - public static CreateTransaction createTransaction() { - return new AutoValue_LocalSpannerIO_CreateTransaction.Builder() - .setSpannerConfig(SpannerConfig.create()) - .setTimestampBound(TimestampBound.strong()) - .build(); - } - - /** - * Creates an uninitialized instance of {@link Write}. Before use, the {@link Write} must be - * configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify - * the Cloud Spanner database being written. - */ - @Experimental - public static Write write() { - return new AutoValue_LocalSpannerIO_Write.Builder() - .setSpannerConfig(SpannerConfig.create()) - .setBatchSizeBytes(DEFAULT_BATCH_SIZE_BYTES) - .setMaxNumMutations(DEFAULT_MAX_NUM_MUTATIONS) - .setMaxNumRows(DEFAULT_MAX_NUM_ROWS) - .setFailureMode(FailureMode.FAIL_FAST) - .build(); - } - - /** Implementation of {@link #readAll}. */ - @AutoValue - public abstract static class ReadAll - extends PTransform, PCollection> { - - abstract SpannerConfig getSpannerConfig(); - - @Nullable - abstract PCollectionView getTransaction(); - - @Nullable - abstract TimestampBound getTimestampBound(); - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setSpannerConfig(SpannerConfig spannerConfig); - - abstract Builder setTransaction(PCollectionView transaction); - - abstract Builder setTimestampBound(TimestampBound timestampBound); - - abstract Builder setBatching(Boolean batching); - - abstract ReadAll build(); - } - - /** Specifies the Cloud Spanner configuration. */ - public ReadAll withSpannerConfig(SpannerConfig spannerConfig) { - return toBuilder().setSpannerConfig(spannerConfig).build(); - } - - /** Specifies the Cloud Spanner project. */ - public ReadAll withProjectId(String projectId) { - return withProjectId(ValueProvider.StaticValueProvider.of(projectId)); - } - - /** Specifies the Cloud Spanner project. */ - public ReadAll withProjectId(ValueProvider projectId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withProjectId(projectId)); - } - - /** Specifies the Cloud Spanner instance. */ - public ReadAll withInstanceId(String instanceId) { - return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId)); - } - - /** Specifies the Cloud Spanner instance. */ - public ReadAll withInstanceId(ValueProvider instanceId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withInstanceId(instanceId)); - } - - /** Specifies the Cloud Spanner database. */ - public ReadAll withDatabaseId(String databaseId) { - return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId)); - } - - /** Specifies the Cloud Spanner host. */ - public ReadAll withHost(ValueProvider host) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withHost(host)); - } - - public ReadAll withHost(String host) { - return withHost(ValueProvider.StaticValueProvider.of(host)); - } - - /** Specifies the Cloud Spanner database. */ - public ReadAll withDatabaseId(ValueProvider databaseId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withDatabaseId(databaseId)); - } - - @VisibleForTesting - ReadAll withServiceFactory(ServiceFactory serviceFactory) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withServiceFactory(serviceFactory)); - } - - public ReadAll withTransaction(PCollectionView transaction) { - return toBuilder().setTransaction(transaction).build(); - } - - public ReadAll withTimestamp(Timestamp timestamp) { - return withTimestampBound(TimestampBound.ofReadTimestamp(timestamp)); - } - - public ReadAll withTimestampBound(TimestampBound timestampBound) { - return toBuilder().setTimestampBound(timestampBound).build(); - } - - /** - * By default Batch API is used to read data from Cloud Spanner. It is useful to disable - * batching when the underlying query is not root-partitionable. - */ - public ReadAll withBatching(boolean batching) { - return toBuilder().setBatching(batching).build(); - } - - abstract Boolean getBatching(); - - @Override - public PCollection expand(PCollection input) { - PTransform, PCollection> readTransform; - if (getBatching()) { - readTransform = - LocalBatchSpannerRead.create(getSpannerConfig(), getTransaction(), getTimestampBound()); - } else { - readTransform = - NaiveSpannerRead.create(getSpannerConfig(), getTransaction(), getTimestampBound()); - } - return input - .apply("Reshuffle", Reshuffle.viaRandomKey()) - .apply("Read from Cloud Spanner", readTransform); - } - } - - /** Implementation of {@link #read}. */ - @AutoValue - public abstract static class Read extends PTransform> { - - abstract SpannerConfig getSpannerConfig(); - - abstract ReadOperation getReadOperation(); - - @Nullable - abstract TimestampBound getTimestampBound(); - - @Nullable - abstract PCollectionView getTransaction(); - - @Nullable - abstract PartitionOptions getPartitionOptions(); - - abstract Boolean getBatching(); - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - - abstract Builder setSpannerConfig(SpannerConfig spannerConfig); - - abstract Builder setReadOperation(ReadOperation readOperation); - - abstract Builder setTimestampBound(TimestampBound timestampBound); - - abstract Builder setTransaction(PCollectionView transaction); - - abstract Builder setPartitionOptions(PartitionOptions partitionOptions); - - abstract Builder setBatching(Boolean batching); - - abstract Read build(); - } - - /** Specifies the Cloud Spanner configuration. */ - public Read withSpannerConfig(SpannerConfig spannerConfig) { - return toBuilder().setSpannerConfig(spannerConfig).build(); - } - - /** Specifies the Cloud Spanner project. */ - public Read withProjectId(String projectId) { - return withProjectId(ValueProvider.StaticValueProvider.of(projectId)); - } - - /** Specifies the Cloud Spanner project. */ - public Read withProjectId(ValueProvider projectId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withProjectId(projectId)); - } - - /** Specifies the Cloud Spanner instance. */ - public Read withInstanceId(String instanceId) { - return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId)); - } - - /** Specifies the Cloud Spanner instance. */ - public Read withInstanceId(ValueProvider instanceId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withInstanceId(instanceId)); - } - - /** Specifies the Cloud Spanner database. */ - public Read withDatabaseId(String databaseId) { - return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId)); - } - - /** Specifies the Cloud Spanner database. */ - public Read withDatabaseId(ValueProvider databaseId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withDatabaseId(databaseId)); - } - - /** Specifies the Cloud Spanner host. */ - public Read withHost(ValueProvider host) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withHost(host)); - } - - public Read withHost(String host) { - return withHost(ValueProvider.StaticValueProvider.of(host)); - } - - /** If true the uses Cloud Spanner batch API. */ - public Read withBatching(boolean batching) { - return toBuilder().setBatching(batching).build(); - } - - @VisibleForTesting - Read withServiceFactory(ServiceFactory serviceFactory) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withServiceFactory(serviceFactory)); - } - - public Read withTransaction(PCollectionView transaction) { - return toBuilder().setTransaction(transaction).build(); - } - - public Read withTimestamp(Timestamp timestamp) { - return withTimestampBound(TimestampBound.ofReadTimestamp(timestamp)); - } - - public Read withTimestampBound(TimestampBound timestampBound) { - return toBuilder().setTimestampBound(timestampBound).build(); - } - - public Read withTable(String table) { - return withReadOperation(getReadOperation().withTable(table)); - } - - public Read withReadOperation(ReadOperation operation) { - return toBuilder().setReadOperation(operation).build(); - } - - public Read withColumns(String... columns) { - return withColumns(Arrays.asList(columns)); - } - - public Read withColumns(List columns) { - return withReadOperation(getReadOperation().withColumns(columns)); - } - - public Read withQuery(Statement statement) { - return withReadOperation(getReadOperation().withQuery(statement)); - } - - public Read withQuery(String sql) { - return withQuery(Statement.of(sql)); - } - - public Read withKeySet(KeySet keySet) { - return withReadOperation(getReadOperation().withKeySet(keySet)); - } - - public Read withIndex(String index) { - return withReadOperation(getReadOperation().withIndex(index)); - } - - public Read withPartitionOptions(PartitionOptions partitionOptions) { - return withReadOperation(getReadOperation().withPartitionOptions(partitionOptions)); - } - - @Override - public PCollection expand(PBegin input) { - getSpannerConfig().validate(); - checkArgument( - getTimestampBound() != null, - "SpannerIO.read() runs in a read only transaction and requires timestamp to be set " - + "with withTimestampBound or withTimestamp method"); - - if (getReadOperation().getQuery() != null) { - // TODO: validate query? - } else if (getReadOperation().getTable() != null) { - // Assume read - checkNotNull( - getReadOperation().getColumns(), - "For a read operation SpannerIO.read() requires a list of " - + "columns to set with withColumns method"); - checkArgument( - !getReadOperation().getColumns().isEmpty(), - "For a read operation SpannerIO.read() requires a" - + " list of columns to set with withColumns method"); - } else { - throw new IllegalArgumentException( - "SpannerIO.read() requires configuring query or read operation."); - } - - ReadAll readAll = - readAll() - .withSpannerConfig(getSpannerConfig()) - .withTimestampBound(getTimestampBound()) - .withBatching(getBatching()) - .withTransaction(getTransaction()); - return input.apply(Create.of(getReadOperation())).apply("Execute query", readAll); - } - } - - /** - * A {@link PTransform} that create a transaction. - * - * @see SpannerIO - */ - @AutoValue - public abstract static class CreateTransaction - extends PTransform> { - - abstract SpannerConfig getSpannerConfig(); - - @Nullable - abstract TimestampBound getTimestampBound(); - - abstract Builder toBuilder(); - - @Override - public PCollectionView expand(PBegin input) { - getSpannerConfig().validate(); - - return input - .apply(Create.of(1)) - .apply("Create transaction", ParDo.of(new LocalCreateTransactionFn(this))) - .apply("As PCollectionView", View.asSingleton()); - } - - /** Specifies the Cloud Spanner configuration. */ - public CreateTransaction withSpannerConfig(SpannerConfig spannerConfig) { - return toBuilder().setSpannerConfig(spannerConfig).build(); - } - - /** Specifies the Cloud Spanner project. */ - public CreateTransaction withProjectId(String projectId) { - return withProjectId(ValueProvider.StaticValueProvider.of(projectId)); - } - - /** Specifies the Cloud Spanner project. */ - public CreateTransaction withProjectId(ValueProvider projectId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withProjectId(projectId)); - } - - /** Specifies the Cloud Spanner instance. */ - public CreateTransaction withInstanceId(String instanceId) { - return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId)); - } - - /** Specifies the Cloud Spanner instance. */ - public CreateTransaction withInstanceId(ValueProvider instanceId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withInstanceId(instanceId)); - } - - /** Specifies the Cloud Spanner database. */ - public CreateTransaction withDatabaseId(String databaseId) { - return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId)); - } - - /** Specifies the Cloud Spanner database. */ - public CreateTransaction withDatabaseId(ValueProvider databaseId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withDatabaseId(databaseId)); - } - - /** Specifies the Cloud Spanner host. */ - public CreateTransaction withHost(ValueProvider host) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withHost(host)); - } - - public CreateTransaction withHost(String host) { - return withHost(ValueProvider.StaticValueProvider.of(host)); - } - - @VisibleForTesting - CreateTransaction withServiceFactory(ServiceFactory serviceFactory) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withServiceFactory(serviceFactory)); - } - - public CreateTransaction withTimestampBound(TimestampBound timestampBound) { - return toBuilder().setTimestampBound(timestampBound).build(); - } - - /** A builder for {@link CreateTransaction}. */ - @AutoValue.Builder - public abstract static class Builder { - - public abstract Builder setSpannerConfig(SpannerConfig spannerConfig); - - public abstract Builder setTimestampBound(TimestampBound newTimestampBound); - - public abstract CreateTransaction build(); - } - } - - /** A failure handling strategy. */ - public enum FailureMode { - /** Invalid write to Spanner will cause the pipeline to fail. A default strategy. */ - FAIL_FAST, - /** Invalid mutations will be returned as part of the result of the write transform. */ - REPORT_FAILURES - } - - /** - * A {@link PTransform} that writes {@link Mutation} objects to Google Cloud Spanner. - * - * @see SpannerIO - */ - @AutoValue - public abstract static class Write extends PTransform, SpannerWriteResult> { - - abstract SpannerConfig getSpannerConfig(); - - abstract long getBatchSizeBytes(); - - abstract long getMaxNumMutations(); - - abstract long getMaxNumRows(); - - abstract FailureMode getFailureMode(); - - @Nullable - abstract PCollection getSchemaReadySignal(); - - abstract OptionalInt getGroupingFactor(); - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - - abstract Builder setSpannerConfig(SpannerConfig spannerConfig); - - abstract Builder setBatchSizeBytes(long batchSizeBytes); - - abstract Builder setMaxNumMutations(long maxNumMutations); - - abstract Builder setMaxNumRows(long maxNumRows); - - abstract Builder setFailureMode(FailureMode failureMode); - - abstract Builder setSchemaReadySignal(PCollection schemaReadySignal); - - abstract Builder setGroupingFactor(int groupingFactor); - - abstract Write build(); - } - - /** Specifies the Cloud Spanner configuration. */ - public Write withSpannerConfig(SpannerConfig spannerConfig) { - return toBuilder().setSpannerConfig(spannerConfig).build(); - } - - /** Specifies the Cloud Spanner project. */ - public Write withProjectId(String projectId) { - return withProjectId(ValueProvider.StaticValueProvider.of(projectId)); - } - - /** Specifies the Cloud Spanner project. */ - public Write withProjectId(ValueProvider projectId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withProjectId(projectId)); - } - - /** Specifies the Cloud Spanner instance. */ - public Write withInstanceId(String instanceId) { - return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId)); - } - - /** Specifies the Cloud Spanner instance. */ - public Write withInstanceId(ValueProvider instanceId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withInstanceId(instanceId)); - } - - /** Specifies the Cloud Spanner database. */ - public Write withDatabaseId(String databaseId) { - return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId)); - } - - /** Specifies the Cloud Spanner database. */ - public Write withDatabaseId(ValueProvider databaseId) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withDatabaseId(databaseId)); - } - - /** Specifies the Cloud Spanner host. */ - public Write withHost(ValueProvider host) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withHost(host)); - } - - /** Specifies the Cloud Spanner host. */ - public Write withHost(String host) { - return withHost(ValueProvider.StaticValueProvider.of(host)); - } - - /** - * Specifies the deadline for the Commit API call. Default is 15 secs. DEADLINE_EXCEEDED errors - * will prompt a backoff/retry until the value of {@link #withMaxCumulativeBackoff(Duration)} is - * reached. DEADLINE_EXCEEDED errors are are reported with logging and counters. - */ - public Write withCommitDeadline(Duration commitDeadline) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withCommitDeadline(commitDeadline)); - } - - /** - * Specifies the maximum cumulative backoff time when retrying after DEADLINE_EXCEEDED errors. - * Default is 15 mins. - * - *

If the mutations still have not been written after this time, they are treated as a - * failure, and handled according to the setting of {@link #withFailureMode(FailureMode)}. - */ - public Write withMaxCumulativeBackoff(Duration maxCumulativeBackoff) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withMaxCumulativeBackoff(maxCumulativeBackoff)); - } - - @VisibleForTesting - Write withServiceFactory(ServiceFactory serviceFactory) { - SpannerConfig config = getSpannerConfig(); - return withSpannerConfig(config.withServiceFactory(serviceFactory)); - } - - /** Same transform but can be applied to {@link PCollection} of {@link MutationGroup}. */ - public WriteGrouped grouped() { - return new WriteGrouped(this); - } - - /** - * Specifies the batch size limit (max number of bytes mutated per batch). Default value is 1MB - */ - public Write withBatchSizeBytes(long batchSizeBytes) { - return toBuilder().setBatchSizeBytes(batchSizeBytes).build(); - } - - /** Specifies failure mode. {@link FailureMode#FAIL_FAST} mode is selected by default. */ - public Write withFailureMode(FailureMode failureMode) { - return toBuilder().setFailureMode(failureMode).build(); - } - - /** - * Specifies the cell mutation limit (maximum number of mutated cells per batch). Default value - * is 5000 - */ - public Write withMaxNumMutations(long maxNumMutations) { - return toBuilder().setMaxNumMutations(maxNumMutations).build(); - } - - /** - * Specifies the row mutation limit (maximum number of mutated rows per batch). Default value is - * 1000 - */ - public Write withMaxNumRows(long maxNumRows) { - return toBuilder().setMaxNumRows(maxNumRows).build(); - } - - /** - * Specifies an optional input PCollection that can be used as the signal for {@link - * Wait.OnSignal} to indicate when the database schema is ready to be read. - * - *

To be used when the database schema is created by another section of the pipeline, this - * causes this transform to wait until the {@code signal PCollection} has been closed before - * reading the schema from the database. - * - * @see Wait.OnSignal - */ - public Write withSchemaReadySignal(PCollection signal) { - return toBuilder().setSchemaReadySignal(signal).build(); - } - - /** - * Specifies the multiple of max mutation (in terms of both bytes per batch and cells per batch) - * that is used to select a set of mutations to sort by key for batching. This sort uses local - * memory on the workers, so using large values can cause out of memory errors. Default value is - * 1000. - */ - public Write withGroupingFactor(int groupingFactor) { - return toBuilder().setGroupingFactor(groupingFactor).build(); - } - - @Override - public SpannerWriteResult expand(PCollection input) { - getSpannerConfig().validate(); - - return input - .apply("To mutation group", ParDo.of(new ToMutationGroupFn())) - .apply("Write mutations to Cloud Spanner", new WriteGrouped(this)); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - populateDisplayDataWithParamaters(builder); - } - - private void populateDisplayDataWithParamaters(DisplayData.Builder builder) { - getSpannerConfig().populateDisplayData(builder); - builder.add( - DisplayData.item("batchSizeBytes", getBatchSizeBytes()) - .withLabel("Max batch size in bytes")); - builder.add( - DisplayData.item("maxNumMutations", getMaxNumMutations()) - .withLabel("Max number of mutated cells in each batch")); - builder.add( - DisplayData.item("maxNumRows", getMaxNumRows()) - .withLabel("Max number of rows in each batch")); - // Grouping factor default value depends on whether it is a batch or streaming pipeline. - // This function is not aware of that state, so use 'DEFAULT' if unset. - builder.add( - DisplayData.item( - "groupingFactor", - (getGroupingFactor().isPresent() - ? Integer.toString(getGroupingFactor().getAsInt()) - : "DEFAULT")) - .withLabel("Number of batches to sort over")); - } - } - - /** - * A singleton to compare encoded MutationGroups by encoded Key that wraps {@code - * UnsignedBytes#lexicographicalComparator} which unfortunately is not serializable. - */ - private enum EncodedKvMutationGroupComparator - implements Comparator>, Serializable { - INSTANCE { - @Override - public int compare(KV a, KV b) { - return UnsignedBytes.lexicographicalComparator().compare(a.getKey(), b.getKey()); - } - } - } - - /** Same as {@link Write} but supports grouped mutations. */ - public static class WriteGrouped - extends PTransform, SpannerWriteResult> { - private final Write spec; - private static final TupleTag BATCHABLE_MUTATIONS_TAG = - new TupleTag("batchableMutations") {}; - private static final TupleTag> UNBATCHABLE_MUTATIONS_TAG = - new TupleTag>("unbatchableMutations") {}; - - private static final TupleTag MAIN_OUT_TAG = new TupleTag("mainOut") {}; - private static final TupleTag FAILED_MUTATIONS_TAG = - new TupleTag("failedMutations") {}; - private static final SerializableCoder CODER = - SerializableCoder.of(MutationGroup.class); - - public WriteGrouped(Write spec) { - this.spec = spec; - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - spec.populateDisplayDataWithParamaters(builder); - } - - @Override - public SpannerWriteResult expand(PCollection input) { - PCollection> batches; - - if (spec.getBatchSizeBytes() <= 1 - || spec.getMaxNumMutations() <= 1 - || spec.getMaxNumRows() <= 1) { - LOG.info("Batching of mutationGroups is disabled"); - TypeDescriptor> descriptor = - new TypeDescriptor>() {}; - batches = - input.apply(MapElements.into(descriptor).via(element -> ImmutableList.of(element))); - } else { - - // First, read the Cloud Spanner schema. - PCollection schemaSeed = - input.getPipeline().apply("Create Seed", Create.of((Void) null)); - if (spec.getSchemaReadySignal() != null) { - // Wait for external signal before reading schema. - schemaSeed = schemaSeed.apply("Wait for schema", Wait.on(spec.getSchemaReadySignal())); - } - final PCollectionView schemaView = - schemaSeed - .apply( - "Read information schema", - ParDo.of(new LocalReadSpannerSchema(spec.getSpannerConfig()))) - .apply("Schema View", View.asSingleton()); - - // Split the mutations into batchable and unbatchable mutations. - // Filter out mutation groups too big to be batched. - PCollectionTuple filteredMutations = - input - .apply( - "RewindowIntoGlobal", - Window.into(new GlobalWindows()) - .triggering(DefaultTrigger.of()) - .discardingFiredPanes()) - .apply( - "Filter Unbatchable Mutations", - ParDo.of( - new BatchableMutationFilterFn( - schemaView, - UNBATCHABLE_MUTATIONS_TAG, - spec.getBatchSizeBytes(), - spec.getMaxNumMutations(), - spec.getMaxNumRows())) - .withSideInputs(schemaView) - .withOutputTags( - BATCHABLE_MUTATIONS_TAG, TupleTagList.of(UNBATCHABLE_MUTATIONS_TAG))); - - // Build a set of Mutation groups from the current bundle, - // sort them by table/key then split into batches. - PCollection> batchedMutations = - filteredMutations - .get(BATCHABLE_MUTATIONS_TAG) - .apply( - "Gather And Sort", - ParDo.of( - new GatherBundleAndSortFn( - spec.getBatchSizeBytes(), - spec.getMaxNumMutations(), - spec.getMaxNumRows(), - // Do not group on streaming unless explicitly set. - spec.getGroupingFactor() - .orElse( - input.isBounded() == IsBounded.BOUNDED - ? DEFAULT_GROUPING_FACTOR - : 1), - schemaView)) - .withSideInputs(schemaView)) - .apply( - "Create Batches", - ParDo.of( - new BatchFn( - spec.getBatchSizeBytes(), - spec.getMaxNumMutations(), - spec.getMaxNumRows(), - schemaView)) - .withSideInputs(schemaView)); - - // Merge the batched and unbatchable mutation PCollections and write to Spanner. - batches = - PCollectionList.of(filteredMutations.get(UNBATCHABLE_MUTATIONS_TAG)) - .and(batchedMutations) - .apply("Merge", Flatten.pCollections()); - } - - PCollectionTuple result = - batches.apply( - "Write batches to Spanner", - ParDo.of( - new WriteToSpannerFn( - spec.getSpannerConfig(), spec.getFailureMode(), FAILED_MUTATIONS_TAG)) - .withOutputTags(MAIN_OUT_TAG, TupleTagList.of(FAILED_MUTATIONS_TAG))); - - return new SpannerWriteResult( - input.getPipeline(), - result.get(MAIN_OUT_TAG), - result.get(FAILED_MUTATIONS_TAG), - FAILED_MUTATIONS_TAG); - } - - @VisibleForTesting - static MutationGroup decode(byte[] bytes) { - ByteArrayInputStream bis = new ByteArrayInputStream(bytes); - try { - return CODER.decode(bis); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @VisibleForTesting - static byte[] encode(MutationGroup g) { - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - try { - CODER.encode(g, bos); - } catch (IOException e) { - throw new RuntimeException(e); - } - return bos.toByteArray(); - } - } - - private static class ToMutationGroupFn extends DoFn { - @ProcessElement - public void processElement(ProcessContext c) { - Mutation value = c.element(); - c.output(MutationGroup.create(value)); - } - } - - /** - * Gathers a set of mutations together, gets the keys, encodes them to byte[], sorts them and then - * outputs the encoded sorted list. - * - *

Testing notes: With very small amounts of data, each mutation group is in a separate bundle, - * and as batching and sorting is over the bundle, this effectively means that no batching will - * occur, Therefore this DoFn has to be tested in isolation. - */ - @VisibleForTesting - static class GatherBundleAndSortFn extends DoFn>> { - private final long maxBatchSizeBytes; - private final long maxNumMutations; - private final long maxNumRows; - - // total size of the current batch. - private long batchSizeBytes; - // total number of mutated cells. - private long batchCells; - // total number of rows mutated. - private long batchRows; - - private final PCollectionView schemaView; - - private transient ArrayList> mutationsToSort = null; - - GatherBundleAndSortFn( - long maxBatchSizeBytes, - long maxNumMutations, - long maxNumRows, - long groupingFactor, - PCollectionView schemaView) { - this.maxBatchSizeBytes = maxBatchSizeBytes * groupingFactor; - this.maxNumMutations = maxNumMutations * groupingFactor; - this.maxNumRows = maxNumRows * groupingFactor; - this.schemaView = schemaView; - } - - @StartBundle - public synchronized void startBundle() throws Exception { - if (mutationsToSort == null) { - initSorter(); - } else { - throw new IllegalStateException("Sorter should be null here"); - } - } - - private void initSorter() { - mutationsToSort = new ArrayList>((int) maxNumMutations); - batchSizeBytes = 0; - batchCells = 0; - batchRows = 0; - } - - @FinishBundle - public synchronized void finishBundle(FinishBundleContext c) throws Exception { - // Only output when there is something in the batch. - if (mutationsToSort.isEmpty()) { - mutationsToSort = null; - } else { - c.output(sortAndGetList(), Instant.now(), GlobalWindow.INSTANCE); - } - } - - private Iterable> sortAndGetList() throws IOException { - mutationsToSort.sort(EncodedKvMutationGroupComparator.INSTANCE); - ArrayList> tmp = mutationsToSort; - // Ensure no more mutations can be added. - mutationsToSort = null; - return tmp; - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - SpannerSchema spannerSchema = c.sideInput(schemaView); - MutationKeyEncoder encoder = new MutationKeyEncoder(spannerSchema); - MutationGroup mg = c.element(); - long groupSize = MutationSizeEstimator.sizeOf(mg); - long groupCells = MutationCellCounter.countOf(spannerSchema, mg); - long groupRows = mg.size(); - - synchronized (this) { - if (((batchCells + groupCells) > maxNumMutations) - || (batchSizeBytes + groupSize) > maxBatchSizeBytes - || (batchRows + groupRows) > maxNumRows) { - c.output(sortAndGetList()); - initSorter(); - } - - mutationsToSort.add(KV.of(encoder.encodeTableNameAndKey(mg.primary()), encode(mg))); - batchSizeBytes += groupSize; - batchCells += groupCells; - batchRows += groupRows; - } - } - } - - /** Batches mutations together. */ - @VisibleForTesting - static class BatchFn extends DoFn>, Iterable> { - - private final long maxBatchSizeBytes; - private final long maxNumMutations; - private final long maxNumRows; - private final PCollectionView schemaView; - - BatchFn( - long maxBatchSizeBytes, - long maxNumMutations, - long maxNumRows, - PCollectionView schemaView) { - this.maxBatchSizeBytes = maxBatchSizeBytes; - this.maxNumMutations = maxNumMutations; - this.maxNumRows = maxNumRows; - this.schemaView = schemaView; - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - SpannerSchema spannerSchema = c.sideInput(schemaView); - // Current batch of mutations to be written. - ImmutableList.Builder batch = ImmutableList.builder(); - // total size of the current batch. - long batchSizeBytes = 0; - // total number of mutated cells. - long batchCells = 0; - // total number of rows mutated. - long batchRows = 0; - - // Iterate through list, outputting whenever a batch is complete. - for (KV kv : c.element()) { - MutationGroup mg = decode(kv.getValue()); - - long groupSize = MutationSizeEstimator.sizeOf(mg); - long groupCells = MutationCellCounter.countOf(spannerSchema, mg); - long groupRows = mg.size(); - - if (((batchCells + groupCells) > maxNumMutations) - || ((batchSizeBytes + groupSize) > maxBatchSizeBytes - || (batchRows + groupRows > maxNumRows))) { - // Batch is full: output and reset. - c.output(batch.build()); - batch = ImmutableList.builder(); - batchSizeBytes = 0; - batchCells = 0; - batchRows = 0; - } - batch.add(mg); - batchSizeBytes += groupSize; - batchCells += groupCells; - batchRows += groupRows; - } - // End of list, output what is left. - if (batchCells > 0) { - c.output(batch.build()); - } - } - } - - /** - * Filters MutationGroups larger than the batch size to the output tagged with {@code - * UNBATCHABLE_MUTATIONS_TAG}. - * - *

Testing notes: As batching does not occur during full pipline testing, this DoFn must be - * tested in isolation. - */ - @VisibleForTesting - static class BatchableMutationFilterFn extends DoFn { - - private final PCollectionView schemaView; - private final TupleTag> unbatchableMutationsTag; - private final long batchSizeBytes; - private final long maxNumMutations; - private final long maxNumRows; - private final Counter batchableMutationGroupsCounter = - Metrics.counter(WriteGrouped.class, "batchable_mutation_groups"); - private final Counter unBatchableMutationGroupsCounter = - Metrics.counter(WriteGrouped.class, "unbatchable_mutation_groups"); - - BatchableMutationFilterFn( - PCollectionView schemaView, - TupleTag> unbatchableMutationsTag, - long batchSizeBytes, - long maxNumMutations, - long maxNumRows) { - this.schemaView = schemaView; - this.unbatchableMutationsTag = unbatchableMutationsTag; - this.batchSizeBytes = batchSizeBytes; - this.maxNumMutations = maxNumMutations; - this.maxNumRows = maxNumRows; - } - - @DoFn.ProcessElement - public void processElement(ProcessContext c) { - MutationGroup mg = c.element(); - if (mg.primary().getOperation() == Op.DELETE && !isPointDelete(mg.primary())) { - // Ranged deletes are not batchable. - c.output(unbatchableMutationsTag, Arrays.asList(mg)); - unBatchableMutationGroupsCounter.inc(); - return; - } - - SpannerSchema spannerSchema = c.sideInput(schemaView); - long groupSize = MutationSizeEstimator.sizeOf(mg); - long groupCells = MutationCellCounter.countOf(spannerSchema, mg); - long groupRows = Iterables.size(mg); - - if (groupSize >= batchSizeBytes || groupCells >= maxNumMutations || groupRows >= maxNumRows) { - c.output(unbatchableMutationsTag, Arrays.asList(mg)); - unBatchableMutationGroupsCounter.inc(); - } else { - c.output(mg); - batchableMutationGroupsCounter.inc(); - } - } - } - - @VisibleForTesting - static class WriteToSpannerFn extends DoFn, Void> { - - private transient ExposedSpannerAccessor spannerAccessor; - private final SpannerConfig spannerConfig; - private final FailureMode failureMode; - - /* Number of times an aborted write to spanner could be retried */ - private static final int ABORTED_RETRY_ATTEMPTS = 5; - /* Error string in Aborted exception during schema change */ - private final String errString = - "Transaction aborted. " - + "Database schema probably changed during transaction, retry may succeed."; - - @VisibleForTesting static Sleeper sleeper = Sleeper.DEFAULT; - - private final Counter mutationGroupBatchesReceived = - Metrics.counter(WriteGrouped.class, "mutation_group_batches_received"); - private final Counter mutationGroupBatchesWriteSuccess = - Metrics.counter(WriteGrouped.class, "mutation_group_batches_write_success"); - private final Counter mutationGroupBatchesWriteFail = - Metrics.counter(WriteGrouped.class, "mutation_group_batches_write_fail"); - - private final Counter mutationGroupsReceived = - Metrics.counter(WriteGrouped.class, "mutation_groups_received"); - private final Counter mutationGroupsWriteSuccess = - Metrics.counter(WriteGrouped.class, "mutation_groups_write_success"); - private final Counter mutationGroupsWriteFail = - Metrics.counter(WriteGrouped.class, "mutation_groups_write_fail"); - - private final Counter spannerWriteSuccess = - Metrics.counter(WriteGrouped.class, "spanner_write_success"); - private final Counter spannerWriteFail = - Metrics.counter(WriteGrouped.class, "spanner_write_fail"); - private final Distribution spannerWriteLatency = - Metrics.distribution(WriteGrouped.class, "spanner_write_latency_ms"); - private final Counter spannerWriteTimeouts = - Metrics.counter(WriteGrouped.class, "spanner_write_timeouts"); - private final Counter spannerWriteRetries = - Metrics.counter(WriteGrouped.class, "spanner_write_retries"); - - private final TupleTag failedTag; - - private FluentBackoff bundleWriteBackoff; - - WriteToSpannerFn( - SpannerConfig spannerConfig, FailureMode failureMode, TupleTag failedTag) { - this.spannerConfig = spannerConfig; - this.failureMode = failureMode; - this.failedTag = failedTag; - } - - @Setup - public void setup() throws Exception { - // set up non-serializable values here. - spannerAccessor = ExposedSpannerAccessor.create(spannerConfig); - bundleWriteBackoff = - FluentBackoff.DEFAULT - .withMaxCumulativeBackoff(spannerConfig.getMaxCumulativeBackoff().get()) - .withInitialBackoff(spannerConfig.getMaxCumulativeBackoff().get().dividedBy(60)); - } - - @Teardown - public void teardown() throws Exception { - spannerAccessor.close(); - } - - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - Iterable mutations = c.element(); - - // Batch upsert rows. - try { - mutationGroupBatchesReceived.inc(); - mutationGroupsReceived.inc(Iterables.size(mutations)); - Iterable batch = Iterables.concat(mutations); - writeMutations(batch); - mutationGroupBatchesWriteSuccess.inc(); - mutationGroupsWriteSuccess.inc(Iterables.size(mutations)); - return; - } catch (SpannerException e) { - mutationGroupBatchesWriteFail.inc(); - if (failureMode == FailureMode.REPORT_FAILURES) { - // fall through and retry individual mutationGroups. - } else if (failureMode == FailureMode.FAIL_FAST) { - mutationGroupsWriteFail.inc(Iterables.size(mutations)); - throw e; - } else { - throw new IllegalArgumentException("Unknown failure mode " + failureMode); - } - } - - // If we are here, writing a batch has failed, retry individual mutations. - for (MutationGroup mg : mutations) { - try { - spannerWriteRetries.inc(); - writeMutations(mg); - mutationGroupsWriteSuccess.inc(); - } catch (SpannerException e) { - mutationGroupsWriteFail.inc(); - LOG.warn("Failed to write the mutation group: " + mg, e); - c.output(failedTag, mg); - } - } - } - - /* - Spanner aborts all inflight transactions during a schema change. Client is expected - to retry silently. These must not be counted against retry backoff. - */ - private void spannerWriteWithRetryIfSchemaChange(Iterable batch) - throws SpannerException { - for (int retry = 1; ; retry++) { - try { - if (spannerConfig.getRpcPriority() != null - && spannerConfig.getRpcPriority().get() != null) { - spannerAccessor - .getDatabaseClient() - .writeAtLeastOnceWithOptions( - batch, Options.priority(spannerConfig.getRpcPriority().get())); - } else { - spannerAccessor.getDatabaseClient().writeAtLeastOnce(batch); - } - return; - } catch (AbortedException e) { - if (retry >= ABORTED_RETRY_ATTEMPTS) { - throw e; - } - if (e.isRetryable() || e.getMessage().contains(errString)) { - continue; - } - throw e; - } - } - } - - /** Write the Mutations to Spanner, handling DEADLINE_EXCEEDED with backoff/retries. */ - private void writeMutations(Iterable mutations) throws SpannerException, IOException { - BackOff backoff = bundleWriteBackoff.backoff(); - long mutationsSize = Iterables.size(mutations); - - while (true) { - Stopwatch timer = Stopwatch.createStarted(); - // loop is broken on success, timeout backoff/retry attempts exceeded, or other failure. - try { - spannerWriteWithRetryIfSchemaChange(mutations); - spannerWriteSuccess.inc(); - return; - } catch (SpannerException exception) { - if (exception.getErrorCode() == ErrorCode.DEADLINE_EXCEEDED) { - spannerWriteTimeouts.inc(); - - // Potentially backoff/retry after DEADLINE_EXCEEDED. - long sleepTimeMsecs = backoff.nextBackOffMillis(); - if (sleepTimeMsecs == BackOff.STOP) { - LOG.error( - "DEADLINE_EXCEEDED writing batch of {} mutations to Cloud Spanner. " - + "Aborting after too many retries.", - mutationsSize); - spannerWriteFail.inc(); - throw exception; - } - LOG.info( - "DEADLINE_EXCEEDED writing batch of {} mutations to Cloud Spanner, " - + "retrying after backoff of {}ms\n" - + "({})", - mutationsSize, - sleepTimeMsecs, - exception.getMessage()); - spannerWriteRetries.inc(); - try { - sleeper.sleep(sleepTimeMsecs); - } catch (InterruptedException e) { - // ignore. - } - } else { - // Some other failure: pass up the stack. - spannerWriteFail.inc(); - throw exception; - } - } finally { - spannerWriteLatency.update(timer.elapsed(TimeUnit.MILLISECONDS)); - } - } - } - } - - private LocalSpannerIO() {} // Prevent construction. -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java deleted file mode 100644 index 63ce6ee645..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Copyright (C) 2021 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 org.apache.beam.sdk.io.gcp.spanner; - -import com.google.auto.value.AutoValue; -import com.google.cloud.spanner.KeySet; -import com.google.cloud.spanner.PartitionOptions; -import com.google.cloud.spanner.Statement; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import javax.annotation.Nullable; - -/** Encapsulates a spanner read operation. */ -@AutoValue -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) -public abstract class ReadOperation implements Serializable { - - public static ReadOperation create() { - return new AutoValue_ReadOperation.Builder() - .setPartitionOptions(PartitionOptions.getDefaultInstance()) - .setKeySet(KeySet.all()) - .build(); - } - - public abstract @Nullable Statement getQuery(); - - public abstract @Nullable String getTable(); - - public abstract @Nullable String getIndex(); - - public abstract @Nullable List getColumns(); - - public abstract @Nullable KeySet getKeySet(); - - abstract @Nullable PartitionOptions getPartitionOptions(); - - @AutoValue.Builder - abstract static class Builder { - - abstract Builder setQuery(Statement statement); - - abstract Builder setTable(String table); - - abstract Builder setIndex(String index); - - abstract Builder setColumns(List columns); - - abstract Builder setKeySet(KeySet keySet); - - abstract Builder setPartitionOptions(PartitionOptions partitionOptions); - - abstract ReadOperation build(); - } - - abstract Builder toBuilder(); - - public ReadOperation withTable(String table) { - return toBuilder().setTable(table).build(); - } - - public ReadOperation withColumns(String... columns) { - return withColumns(Arrays.asList(columns)); - } - - public ReadOperation withColumns(List columns) { - return toBuilder().setColumns(columns).build(); - } - - public ReadOperation withQuery(Statement statement) { - return toBuilder().setQuery(statement).build(); - } - - public ReadOperation withQuery(String sql) { - return withQuery(Statement.of(sql)); - } - - public ReadOperation withKeySet(KeySet keySet) { - return toBuilder().setKeySet(keySet).build(); - } - - public ReadOperation withIndex(String index) { - return toBuilder().setIndex(index).build(); - } - - public ReadOperation withPartitionOptions(PartitionOptions partitionOptions) { - return toBuilder().setPartitionOptions(partitionOptions).build(); - } -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java deleted file mode 100644 index e1c1a85810..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Copyright (C) 2021 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 org.apache.beam.sdk.io.gcp.spanner; - -import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.auto.value.AutoValue; -import com.google.cloud.ServiceFactory; -import com.google.cloud.spanner.Options.RpcPriority; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerOptions; -import java.io.Serializable; -import javax.annotation.Nullable; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.joda.time.Duration; - -/** Configuration for a Cloud Spanner client. */ -@AutoValue -@SuppressWarnings({ - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) -public abstract class SpannerConfig implements Serializable { - // A default host name for batch traffic. - private static final String DEFAULT_HOST = "https://batch-spanner.googleapis.com/"; - // Deadline for Commit API call. - private static final Duration DEFAULT_COMMIT_DEADLINE = Duration.standardSeconds(15); - // Total allowable backoff time. - private static final Duration DEFAULT_MAX_CUMULATIVE_BACKOFF = Duration.standardMinutes(15); - // A default priority for batch traffic. - private static final RpcPriority DEFAULT_RPC_PRIORITY = RpcPriority.MEDIUM; - - public abstract @Nullable ValueProvider getProjectId(); - - public abstract @Nullable ValueProvider getInstanceId(); - - public abstract @Nullable ValueProvider getDatabaseId(); - - public abstract @Nullable ValueProvider getHost(); - - public abstract @Nullable ValueProvider getEmulatorHost(); - - public abstract @Nullable ValueProvider getCommitDeadline(); - - public abstract @Nullable ValueProvider getMaxCumulativeBackoff(); - - public abstract @Nullable ValueProvider getRpcPriority(); - - @VisibleForTesting - abstract @Nullable ServiceFactory getServiceFactory(); - - abstract Builder toBuilder(); - - public static SpannerConfig create() { - return builder() - .setHost(ValueProvider.StaticValueProvider.of(DEFAULT_HOST)) - .setCommitDeadline(ValueProvider.StaticValueProvider.of(DEFAULT_COMMIT_DEADLINE)) - .setMaxCumulativeBackoff( - ValueProvider.StaticValueProvider.of(DEFAULT_MAX_CUMULATIVE_BACKOFF)) - .setRpcPriority(ValueProvider.StaticValueProvider.of(DEFAULT_RPC_PRIORITY)) - .build(); - } - - static Builder builder() { - return new AutoValue_SpannerConfig.Builder(); - } - - public void validate() { - checkNotNull( - getInstanceId(), - "SpannerIO.read() requires instance id to be set with withInstanceId method"); - checkNotNull( - getDatabaseId(), - "SpannerIO.read() requires database id to be set with withDatabaseId method"); - } - - public void populateDisplayData(DisplayData.Builder builder) { - builder - .addIfNotNull(DisplayData.item("projectId", getProjectId()).withLabel("Output Project")) - .addIfNotNull(DisplayData.item("instanceId", getInstanceId()).withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()).withLabel("Output Database")); - - if (getServiceFactory() != null) { - builder.addIfNotNull( - DisplayData.item("serviceFactory", getServiceFactory().getClass().getName()) - .withLabel("Service Factory")); - } - } - - /** Builder for {@link SpannerConfig}. */ - @AutoValue.Builder - public abstract static class Builder { - - abstract Builder setProjectId(ValueProvider projectId); - - abstract Builder setInstanceId(ValueProvider instanceId); - - abstract Builder setDatabaseId(ValueProvider databaseId); - - abstract Builder setHost(ValueProvider host); - - abstract Builder setEmulatorHost(ValueProvider emulatorHost); - - abstract Builder setCommitDeadline(ValueProvider commitDeadline); - - abstract Builder setMaxCumulativeBackoff(ValueProvider maxCumulativeBackoff); - - abstract Builder setServiceFactory(ServiceFactory serviceFactory); - - abstract Builder setRpcPriority(ValueProvider rpcPriority); - - public abstract SpannerConfig build(); - } - - public SpannerConfig withProjectId(ValueProvider projectId) { - return toBuilder().setProjectId(projectId).build(); - } - - public SpannerConfig withProjectId(String projectId) { - return withProjectId(ValueProvider.StaticValueProvider.of(projectId)); - } - - public SpannerConfig withInstanceId(ValueProvider instanceId) { - return toBuilder().setInstanceId(instanceId).build(); - } - - public SpannerConfig withInstanceId(String instanceId) { - return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId)); - } - - public SpannerConfig withDatabaseId(ValueProvider databaseId) { - return toBuilder().setDatabaseId(databaseId).build(); - } - - public SpannerConfig withDatabaseId(String databaseId) { - return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId)); - } - - public SpannerConfig withHost(ValueProvider host) { - return toBuilder().setHost(host).build(); - } - - public SpannerConfig withEmulatorHost(ValueProvider emulatorHost) { - return toBuilder().setEmulatorHost(emulatorHost).build(); - } - - public SpannerConfig withCommitDeadline(Duration commitDeadline) { - return withCommitDeadline(ValueProvider.StaticValueProvider.of(commitDeadline)); - } - - public SpannerConfig withCommitDeadline(ValueProvider commitDeadline) { - return toBuilder().setCommitDeadline(commitDeadline).build(); - } - - public SpannerConfig withMaxCumulativeBackoff(Duration maxCumulativeBackoff) { - return withMaxCumulativeBackoff(ValueProvider.StaticValueProvider.of(maxCumulativeBackoff)); - } - - public SpannerConfig withMaxCumulativeBackoff(ValueProvider maxCumulativeBackoff) { - return toBuilder().setMaxCumulativeBackoff(maxCumulativeBackoff).build(); - } - - @VisibleForTesting - SpannerConfig withServiceFactory(ServiceFactory serviceFactory) { - return toBuilder().setServiceFactory(serviceFactory).build(); - } - - public SpannerConfig withRpcPriority(ValueProvider rpcPriority) { - return toBuilder().setRpcPriority(rpcPriority).build(); - } - - public SpannerConfig withRpcPriority(RpcPriority rpcPriority) { - return withRpcPriority(ValueProvider.StaticValueProvider.of(rpcPriority)); - } -} diff --git a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java b/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java deleted file mode 100644 index 51dde919cb..0000000000 --- a/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java +++ /dev/null @@ -1,18 +0,0 @@ -/* - * Copyright (C) 2018 Google 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. - */ - -/** Exposed Beam API classes. */ -package org.apache.beam.sdk.io.gcp.spanner; diff --git a/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java b/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java index eff041f7b9..4b65fa8057 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java +++ b/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java @@ -23,9 +23,9 @@ import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Base64; -import org.apache.beam.sdk.io.gcp.spanner.LocalSpannerIO; import org.apache.beam.sdk.io.gcp.spanner.ReadOperation; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Count; @@ -104,7 +104,7 @@ private ReadAllRows(SpannerConfig spanConfig) { @Override public PCollection> expand(PBegin begin) { PCollectionView tx = - begin.apply(LocalSpannerIO.createTransaction().withSpannerConfig(spanConfig)); + begin.apply(SpannerIO.createTransaction().withSpannerConfig(spanConfig)); PCollection sourceDdl = begin.apply("Read Information Schema", new ReadInformationSchema(spanConfig, tx)); @@ -118,7 +118,7 @@ public PCollection> expand(PBegin begin) { PCollection rows = tables.apply( "Read rows from tables", - LocalSpannerIO.readAll().withTransaction(tx).withSpannerConfig(spanConfig)); + SpannerIO.readAll().withTransaction(tx).withSpannerConfig(spanConfig)); return rows.apply( ParDo.of( diff --git a/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java b/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java index e4b998ddca..97bd419323 100644 --- a/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java @@ -32,9 +32,9 @@ import java.util.Collections; import java.util.List; import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.io.gcp.spanner.LocalSpannerIO; import org.apache.beam.sdk.io.gcp.spanner.ReadOperation; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.TestPipeline; @@ -195,7 +195,7 @@ private void exportDbAtTime( .apply("Create export", spannerExport) .apply( "Read all records", - LocalSpannerIO.readAll().withTransaction(tx).withSpannerConfig(sourceConfig)) + SpannerIO.readAll().withTransaction(tx).withSpannerConfig(sourceConfig)) .apply( "Struct To Csv", MapElements.into(TypeDescriptors.strings()) From 72fefb8e35c3133bde63f09e459376e4697185ff Mon Sep 17 00:00:00 2001 From: dhercher Date: Tue, 15 Mar 2022 02:14:56 -0700 Subject: [PATCH 078/145] DatastreamToSQL support for specifying log level and logging SQL in debug mode PiperOrigin-RevId: 434694190 --- .../teleport/v2/cdc/sources/DataStreamIO.java | 32 +++-- .../FormatDatastreamJsonToJson.java | 69 +---------- .../v2/transforms/FormatDatastreamRecord.java | 116 ++++++++++++++++++ .../FormatDatastreamRecordToJson.java | 52 +++----- .../FormatDatastreamJsonToJsonTest.java | 47 +++++-- .../FormatDatastreamRecordToJsonTest.java | 47 +++---- .../v2/templates/DataStreamToPostgres.java | 3 +- .../v2/templates/DataStreamToSQL.java | 4 +- 8 files changed, 225 insertions(+), 145 deletions(-) create mode 100644 v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecord.java diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/sources/DataStreamIO.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/sources/DataStreamIO.java index 2901d03a8a..df53121335 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/sources/DataStreamIO.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/sources/DataStreamIO.java @@ -110,7 +110,8 @@ public class DataStreamIO extends PTransform hashedColumns = new HashMap(); + private Map renameColumns = new HashMap(); + private Boolean hashRowId = false; PCollection directories = null; public DataStreamIO() {} @@ -144,14 +145,20 @@ public DataStreamIO withLowercaseSourceColumns() { } /** - * Add the supplied columnName to the map of column values to be hashed. A new column with a - * hashed value of the first will be created. + * Add the supplied columnName to the map of column values to be renamed. A new column with a + * renamed value of the first will be created. * * @param columnName The column name to look for in the data. - * @param newColumnName The name of the new column created with hashed data. + * @param newColumnName The name of the new column created. */ - public DataStreamIO withHashColumnValue(String columnName, String newColumnName) { - this.hashedColumns.put(columnName, newColumnName); + public DataStreamIO withRenameColumnValue(String columnName, String newColumnName) { + this.renameColumns.put(columnName, newColumnName); + return this; + } + + /** Set the reader to hash Oracle ROWID values into int. */ + public DataStreamIO withHashRowId() { + this.hashRowId = true; return this; } @@ -181,16 +188,19 @@ public PCollection> expandDataStreamJsonStrings( .apply( "ParseJsonRecords", ParDo.of( - FormatDatastreamJsonToJson.create() - .withStreamName(this.streamName) - .withHashColumnValues(this.hashedColumns) - .withLowercaseSourceColumns(this.lowercaseSourceColumns))) + (FormatDatastreamJsonToJson) + FormatDatastreamJsonToJson.create() + .withStreamName(this.streamName) + .withRenameColumnValues(this.renameColumns) + .withHashRowId(this.hashRowId) + .withLowercaseSourceColumns(this.lowercaseSourceColumns))) .setCoder(coder); } else { SerializableFunction> parseFn = FormatDatastreamRecordToJson.create() .withStreamName(this.streamName) - .withHashColumnValues(this.hashedColumns) + .withRenameColumnValues(this.renameColumns) + .withHashRowId(this.hashRowId) .withLowercaseSourceColumns(this.lowercaseSourceColumns); datastreamRecords = datastreamFiles diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java index 1df507bca2..ed76410d74 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java @@ -21,10 +21,7 @@ import java.time.ZoneId; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; -import java.util.HashMap; import java.util.Iterator; -import java.util.Map; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.codehaus.jackson.JsonNode; import org.codehaus.jackson.map.ObjectMapper; @@ -37,15 +34,11 @@ * will use downstream. */ public final class FormatDatastreamJsonToJson - extends DoFn> { + extends FormatDatastreamRecord> { static final Logger LOG = LoggerFactory.getLogger(FormatDatastreamJsonToJson.class); static final DateTimeFormatter DEFAULT_TIMESTAMP_WITH_TZ_FORMATTER = DateTimeFormatter.ISO_OFFSET_DATE_TIME; - private String rowIdColumnName; - private Map hashedColumns = new HashMap(); - private boolean lowercaseSourceColumns = false; - private String streamName; private FormatDatastreamJsonToJson() {} @@ -53,48 +46,6 @@ public static FormatDatastreamJsonToJson create() { return new FormatDatastreamJsonToJson(); } - public FormatDatastreamJsonToJson withStreamName(String streamName) { - this.streamName = streamName; - return this; - } - - public FormatDatastreamJsonToJson withLowercaseSourceColumns(Boolean lowercaseSourceColumns) { - this.lowercaseSourceColumns = lowercaseSourceColumns; - return this; - } - - /** - * Add the supplied columnName to the list of column values to be hashed. - * - * @param columnName The column name to look for in the data to hash. - */ - public FormatDatastreamJsonToJson withHashColumnValue(String columnName) { - this.hashedColumns.put(columnName, columnName); - return this; - } - - /** - * Add the supplied columnName to the map of column values to be hashed. A new column with a - * hashed value of the first will be created. - * - * @param columnName The column name to look for in the data. - * @param newColumnName The name of the new column created with hashed data. - */ - public FormatDatastreamJsonToJson withHashColumnValue(String columnName, String newColumnName) { - this.hashedColumns.put(columnName, newColumnName); - return this; - } - - /** - * Set the map of columns values to hash. - * - * @param hashedColumns The map of columns to new columns to hash. - */ - public FormatDatastreamJsonToJson withHashColumnValues(Map hashedColumns) { - this.hashedColumns = hashedColumns; - return this; - } - @ProcessElement public void processElement(ProcessContext c) { @@ -143,7 +94,7 @@ record = new ObjectMapper().readTree(c.element()); } else { // Oracle Specific Metadata outputObject.put("_metadata_schema", getSourceMetadata(record, "schema")); - outputObject.put("_metadata_row_id", getSourceMetadata(record, "row_id")); + setOracleRowIdValue(outputObject, getSourceMetadata(record, "row_id")); outputObject.put("_metadata_scn", getSourceMetadataAsLong(record, "scn")); outputObject.put("_metadata_ssn", getSourceMetadataAsLong(record, "ssn")); outputObject.put("_metadata_rs_id", getSourceMetadata(record, "rs_id")); @@ -163,10 +114,11 @@ record = new ObjectMapper().readTree(c.element()); outputObject.put(key, payload.get(key)); } } - // Hash columns supplied to be hashed - applyHashToColumns(payload, outputObject); } + // Rename/Copy columns supplied (including _metadata_* columns) + applyRenameColumns(outputObject); + // All Raw Metadata outputObject.put("_metadata_source", getSourceMetadata(record)); @@ -283,15 +235,4 @@ private Boolean getMetadataIsDeleted(JsonNode record) { return value.getBooleanValue(); } - - private void applyHashToColumns(JsonNode record, ObjectNode outputObject) { - for (String columnName : this.hashedColumns.keySet()) { - if (record.get(columnName) != null) { - // TODO: discuss hash algorithm to use - String newColumnName = this.hashedColumns.get(columnName); - int hashedValue = record.get(columnName).getTextValue().hashCode(); - outputObject.put(newColumnName, hashedValue); - } - } - } } diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecord.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecord.java new file mode 100644 index 0000000000..9455533360 --- /dev/null +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecord.java @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2018 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.v2.transforms; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import org.apache.beam.sdk.transforms.DoFn; +import org.codehaus.jackson.node.ObjectNode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** An abstract class to cover the generic requirements Datastream row cleaning processes. */ +public abstract class FormatDatastreamRecord extends DoFn { + + static final Logger LOG = LoggerFactory.getLogger(FormatDatastreamJsonToJson.class); + + protected String streamName; + protected boolean lowercaseSourceColumns = false; + protected Map renameColumns = new HashMap(); + protected boolean hashRowId = false; + + static final String ROW_ID_CHARSET = + "+/0123456789abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ"; + static final Map ROW_ID_CHARSET_MAP = + (Map) + LongStream.range(0, ROW_ID_CHARSET.length()) + .mapToObj(i -> i) + .collect(Collectors.toMap(i -> ROW_ID_CHARSET.charAt(i.intValue()), i -> i)); + + /** + * Set the map of columns values to rename/copy. + * + * @param renameColumns The map of columns to new columns to rename/copy. + */ + public FormatDatastreamRecord withRenameColumnValues(Map renameColumns) { + this.renameColumns = renameColumns; + return this; + } + + public FormatDatastreamRecord withLowercaseSourceColumns(Boolean lowercaseSourceColumns) { + this.lowercaseSourceColumns = lowercaseSourceColumns; + return this; + } + + public FormatDatastreamRecord withStreamName(String streamName) { + this.streamName = streamName; + return this; + } + + /** Set the reader to hash Oracle ROWID values into int. */ + public FormatDatastreamRecord withHashRowId(Boolean hashRowId) { + this.hashRowId = hashRowId; + return this; + } + + protected void applyRenameColumns(ObjectNode outputObject) { + applyRenameColumns(outputObject, this.renameColumns); + } + + protected static void applyRenameColumns( + ObjectNode outputObject, Map renameColumns) { + for (String columnName : renameColumns.keySet()) { + if (outputObject.get(columnName) != null) { + String newColumnName = renameColumns.get(columnName); + outputObject.put(newColumnName, outputObject.get(columnName)); + } + } + } + + protected void setOracleRowIdValue(ObjectNode outputObject, String rowId) { + setOracleRowIdValue(outputObject, rowId, this.hashRowId); + } + + protected static void setOracleRowIdValue( + ObjectNode outputObject, String rowId, Boolean hashRowId) { + if (hashRowId) { + outputObject.put("_metadata_row_id", hashRowIdToInt(rowId)); + } else { + outputObject.put("_metadata_row_id", rowId); + } + } + + /** Hash an Oracle ROWID into a unique identifier for a row which fits in a long. */ + protected static long hashRowIdToInt(String rowId) { + if (rowId == null) { + LOG.warn("Oracle RowId is null: \"{}\"", rowId); + return -1; + } else if (rowId.length() != 18) { + LOG.warn("Oracle RowId Invalid Length: \"{}\" -> {}", rowId, rowId.length()); + return -1; + } else if (!rowId.matches("[a-zA-Z0-9+/]*")) { + LOG.warn("Oracle RowId Invalid Value: \"{}\"", rowId); + return -1; + } + + String rowLocationData = rowId.substring(8); + return LongStream.range(0, rowLocationData.length()) + .map(i -> ROW_ID_CHARSET_MAP.get(rowLocationData.charAt((int) i)) * (long) Math.pow(64, i)) + .sum(); + } +} diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJson.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJson.java index 576d7107db..e1bd422433 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJson.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJson.java @@ -64,7 +64,8 @@ public static class CustomAvroTypes { private String streamName; private boolean lowercaseSourceColumns = false; private String rowIdColumnName; - private Map hashedColumns = new HashMap(); + private Map renameColumns = new HashMap(); + private boolean hashRowId = false; private FormatDatastreamRecordToJson() {} @@ -83,34 +84,18 @@ public FormatDatastreamRecordToJson withLowercaseSourceColumns(Boolean lowercase } /** - * Add the supplied columnName to the list of column values to be hashed. + * Set the map of columns values to rename/copy. * - * @param columnName The column name to look for in the data to hash. + * @param renameColumns The map of columns to new columns to rename/copy. */ - public FormatDatastreamRecordToJson withHashColumnValue(String columnName) { - this.hashedColumns.put(columnName, columnName); + public FormatDatastreamRecordToJson withRenameColumnValues(Map renameColumns) { + this.renameColumns = renameColumns; return this; } - /** - * Add the supplied columnName to the map of column values to be hashed. A new column with a - * hashed value of the first will be created. - * - * @param columnName The column name to look for in the data. - * @param newColumnName The name of the new column created with hashed data. - */ - public FormatDatastreamRecordToJson withHashColumnValue(String columnName, String newColumnName) { - this.hashedColumns.put(columnName, newColumnName); - return this; - } - - /** - * Set the map of columns values to hash. - * - * @param hashedColumns The map of columns to new columns to hash. - */ - public FormatDatastreamRecordToJson withHashColumnValues(Map hashedColumns) { - this.hashedColumns = hashedColumns; + /** Set the reader to hash Oracle ROWID values into int. */ + public FormatDatastreamRecordToJson withHashRowId(Boolean hashRowId) { + this.hashRowId = hashRowId; return this; } @@ -146,14 +131,16 @@ public FailsafeElement apply(GenericRecord record) { } else { // Oracle Specific Metadata outputObject.put("_metadata_schema", getMetadataSchema(record)); - outputObject.put("_metadata_row_id", getOracleRowId(record)); outputObject.put("_metadata_scn", getOracleScn(record)); outputObject.put("_metadata_ssn", getOracleSsn(record)); outputObject.put("_metadata_rs_id", getOracleRsId(record)); outputObject.put("_metadata_tx_id", getOracleTxId(record)); + + FormatDatastreamRecord.setOracleRowIdValue( + outputObject, getOracleRowId(record), this.hashRowId); } - // Hash columns supplied to be hashed - applyHashToColumns(outputObject); + // Rename columns supplied + FormatDatastreamRecord.applyRenameColumns(outputObject, this.renameColumns); // All Raw Metadata outputObject.put("_metadata_source", getSourceMetadataJson(record)); @@ -271,17 +258,6 @@ private String getOracleRowId(GenericRecord record) { return null; } - private void applyHashToColumns(ObjectNode outputObject) { - for (String columnName : this.hashedColumns.keySet()) { - if (outputObject.get(columnName) != null) { - // TODO: discuss hash algorithm to use - String newColumnName = this.hashedColumns.get(columnName); - int hashedValue = outputObject.get(columnName).toString().hashCode(); - outputObject.put(newColumnName, hashedValue); - } - } - } - private Long getOracleScn(GenericRecord record) { if (((GenericRecord) record.get("source_metadata")).get("scn") != null) { return (Long) ((GenericRecord) record.get("source_metadata")).get("scn"); diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java index 2eb9807688..7df363c3c4 100644 --- a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJsonTest.java @@ -17,7 +17,7 @@ import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; import com.google.cloud.teleport.v2.values.FailsafeElement; -import java.util.HashMap; +import com.google.common.collect.ImmutableMap; import java.util.Map; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PAssert; @@ -45,11 +45,16 @@ public class FormatDatastreamJsonToJsonTest { private static final String EXAMPLE_DATASTREAM_RECORD = "{\"_metadata_stream\":\"my-stream\",\"_metadata_timestamp\":1640410924,\"_metadata_read_timestamp\":1640410924,\"_metadata_read_method\":\"oracle-backfill\",\"_metadata_source_type\":\"oracle\",\"_metadata_deleted\":false,\"_metadata_table\":\"JOBS\",\"_metadata_change_type\":\"INSERT\",\"_metadata_primary_keys\":[\"JOB_ID\"],\"_metadata_schema\":\"HR\",\"_metadata_row_id\":\"AAAEARAAEAAAAC9AAS\",\"_metadata_scn\":1706664,\"_metadata_ssn\":0,\"_metadata_rs_id\":\"\",\"_metadata_tx_id\":null,\"JOB_ID\":\"PR_REP\",\"JOB_TITLE\":\"Public" + " Relations" - + " Representative\",\"MIN_SALARY\":4500,\"MAX_SALARY\":10500,\"_metadata_source\":{\"schema\":\"HR\",\"table\":\"JOBS\",\"database\":\"XE\",\"row_id\":\"AAAEARAAEAAAAC9AAS\",\"scn\":1706664,\"is_deleted\":false,\"change_type\":\"INSERT\",\"ssn\":0,\"rs_id\":\"\",\"tx_id\":null,\"log_file\":\"\",\"primary_keys\":[\"JOB_ID\"]}}"; + + " Representative\",\"MIN_SALARY\":4500,\"MAX_SALARY\":10500,\"rowid\":\"AAAEARAAEAAAAC9AAS\",\"_metadata_source\":{\"schema\":\"HR\",\"table\":\"JOBS\",\"database\":\"XE\",\"row_id\":\"AAAEARAAEAAAAC9AAS\",\"scn\":1706664,\"is_deleted\":false,\"change_type\":\"INSERT\",\"ssn\":0,\"rs_id\":\"\",\"tx_id\":null,\"log_file\":\"\",\"primary_keys\":[\"JOB_ID\"]}}"; + + private static final String EXAMPLE_DATASTREAM_RECORD_WITH_HASH_ROWID = + "{\"_metadata_stream\":\"my-stream\",\"_metadata_timestamp\":1640410924,\"_metadata_read_timestamp\":1640410924,\"_metadata_read_method\":\"oracle-backfill\",\"_metadata_source_type\":\"oracle\",\"_metadata_deleted\":false,\"_metadata_table\":\"JOBS\",\"_metadata_change_type\":\"INSERT\",\"_metadata_primary_keys\":[\"JOB_ID\"],\"_metadata_schema\":\"HR\",\"_metadata_row_id\":1019670290924988842,\"_metadata_scn\":1706664,\"_metadata_ssn\":0,\"_metadata_rs_id\":\"\",\"_metadata_tx_id\":null,\"JOB_ID\":\"PR_REP\",\"JOB_TITLE\":\"Public" + + " Relations" + + " Representative\",\"MIN_SALARY\":4500,\"MAX_SALARY\":10500,\"rowid\":1019670290924988842,\"_metadata_source\":{\"schema\":\"HR\",\"table\":\"JOBS\",\"database\":\"XE\",\"row_id\":\"AAAEARAAEAAAAC9AAS\",\"scn\":1706664,\"is_deleted\":false,\"change_type\":\"INSERT\",\"ssn\":0,\"rs_id\":\"\",\"tx_id\":null,\"log_file\":\"\",\"primary_keys\":[\"JOB_ID\"]}}"; @Test public void testProcessElement_validJson() { - Map hashedColumns = new HashMap(); + Map renameColumns = ImmutableMap.of("_metadata_row_id", "rowid"); FailsafeElement expectedElement = FailsafeElement.of(EXAMPLE_DATASTREAM_RECORD, EXAMPLE_DATASTREAM_RECORD); @@ -60,10 +65,38 @@ public void testProcessElement_validJson() { .apply( "FormatDatastreamJsonToJson", ParDo.of( - FormatDatastreamJsonToJson.create() - .withStreamName("my-stream") - .withHashColumnValues(hashedColumns) - .withLowercaseSourceColumns(false))) + (FormatDatastreamJsonToJson) + FormatDatastreamJsonToJson.create() + .withStreamName("my-stream") + .withRenameColumnValues(renameColumns) + .withLowercaseSourceColumns(false))) + .setCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); + + PAssert.that(pCollection).containsInAnyOrder(expectedElement); + + pipeline.run(); + } + + @Test + public void testProcessElement_hashRowId() { + Map renameColumns = ImmutableMap.of("_metadata_row_id", "rowid"); + + FailsafeElement expectedElement = + FailsafeElement.of( + EXAMPLE_DATASTREAM_RECORD_WITH_HASH_ROWID, EXAMPLE_DATASTREAM_RECORD_WITH_HASH_ROWID); + + PCollection> pCollection = + pipeline + .apply("CreateInput", Create.of(EXAMPLE_DATASTREAM_JSON)) + .apply( + "FormatDatastreamJsonToJson", + ParDo.of( + (FormatDatastreamJsonToJson) + FormatDatastreamJsonToJson.create() + .withStreamName("my-stream") + .withRenameColumnValues(renameColumns) + .withHashRowId(true) + .withLowercaseSourceColumns(false))) .setCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); PAssert.that(pCollection).containsInAnyOrder(expectedElement); diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java index 8d8c8b7603..95e7563528 100644 --- a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamRecordToJsonTest.java @@ -34,31 +34,18 @@ public class FormatDatastreamRecordToJsonTest { private static final String EXPECTED_FIRST_RECORD = - "{\"LOCATION_ID\":1000.0," - + "\"STREET_ADDRESS\":\"1297 Via Cola di Rie\"," - + "\"POSTAL_CODE\":\"00989\"," - + "\"CITY\":\"Roma\"," - + "\"STATE_PROVINCE\":null," + "{\"LOCATION_ID\":1000.0,\"STREET_ADDRESS\":\"1297 Via Cola di Rie\"," + + "\"POSTAL_CODE\":\"00989\",\"CITY\":\"Roma\",\"STATE_PROVINCE\":null," + "\"COUNTRY_ID\":\"IT\"," + "\"_metadata_stream\":\"projects/596161805475/locations/us-central1/streams/dylan-stream-20200810test2\"," - + "\"_metadata_timestamp\":1597101230," - + "\"_metadata_read_timestamp\":1597101230," - + "\"_metadata_read_method\":\"oracle_dump\"," - + "\"_metadata_source_type\":\"oracle_dump\"," - + "\"_metadata_deleted\":false," - + "\"_metadata_table\":\"LOCATIONS\"," - + "\"_metadata_change_type\":null," - + "\"_metadata_primary_keys\":null," - + "\"_metadata_schema\":\"HR\"," - + "\"_metadata_row_id\":\"AAAEALAAEAAAACdAAB\"," - + "\"_metadata_scn\":null," - + "\"_metadata_ssn\":null," - + "\"_metadata_rs_id\":null," - + "\"_metadata_tx_id\":null," - + "\"_metadata_source\":{\"schema\":\"HR\"," - + "\"table\":\"LOCATIONS\"," - + "\"database\":\"XE\"," - + "\"row_id\":\"AAAEALAAEAAAACdAAB\"}}"; + + "\"_metadata_timestamp\":1597101230,\"_metadata_read_timestamp\":1597101230," + + "\"_metadata_read_method\":\"oracle_dump\",\"_metadata_source_type\":\"oracle_dump\"," + + "\"_metadata_deleted\":false,\"_metadata_table\":\"LOCATIONS\"," + + "\"_metadata_change_type\":null,\"_metadata_primary_keys\":null," + + "\"_metadata_schema\":\"HR\",\"_metadata_scn\":null,\"_metadata_ssn\":null," + + "\"_metadata_rs_id\":null,\"_metadata_tx_id\":null," + + "\"_metadata_row_id\":\"AAAEALAAEAAAACdAAB\",\"_metadata_source\":{\"schema\":\"HR\"," + + "\"table\":\"LOCATIONS\",\"database\":\"XE\",\"row_id\":\"AAAEALAAEAAAACdAAB\"}}"; private static final String EXPECTED_NUMERIC_RECORD = "{\"id\":2,\"bitty\":0,\"booly\":0,\"tiny\":-1,\"small\":-1,\"medium\":-1," @@ -163,4 +150,18 @@ public void testParseMySQLNumbers() throws IOException, URISyntaxException { String jsonData = FormatDatastreamRecordToJson.create().apply(record).getOriginalPayload(); assertEquals(EXPECTED_NUMERIC_RECORD, jsonData); } + + @Test + public void testHashRowId_valid() { + assertEquals(0L, FormatDatastreamRecord.hashRowIdToInt("AAAAAAAA++++++++++")); + assertEquals(1L, FormatDatastreamRecord.hashRowIdToInt("AAAAAAAA/+++++++++")); + assertEquals(2L, FormatDatastreamRecord.hashRowIdToInt("ABCDE1230+++++++++")); + assertEquals(1152921504606846975L, FormatDatastreamRecord.hashRowIdToInt("AAAAAAAAZZZZZZZZZZ")); + } + + @Test + public void testHashRowId_invalid() { + assertEquals(-1L, FormatDatastreamRecord.hashRowIdToInt("")); + assertEquals(-1L, FormatDatastreamRecord.hashRowIdToInt("ABCD")); + } } diff --git a/v2/datastream-to-postgres/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToPostgres.java b/v2/datastream-to-postgres/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToPostgres.java index 3c26f08109..612543ccb6 100644 --- a/v2/datastream-to-postgres/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToPostgres.java +++ b/v2/datastream-to-postgres/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToPostgres.java @@ -224,7 +224,8 @@ public static PipelineResult run(Options options) { options.getGcsPubSubSubscription(), options.getRfcStartDateTime()) .withLowercaseSourceColumns() - .withHashColumnValue("_metadata_row_id", "rowid")); + .withRenameColumnValue("_metadata_row_id", "rowid") + .withHashRowId()); /* * Stage 2: Write JSON Strings to Postgres Insert Strings diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java index 74e85e98bd..f69ff9b486 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java @@ -281,7 +281,8 @@ public static PipelineResult run(Options options) { options.getGcsPubSubSubscription(), options.getRfcStartDateTime()) .withLowercaseSourceColumns() - .withHashColumnValue("_metadata_row_id", "rowid")); + .withRenameColumnValue("_metadata_row_id", "rowid") + .withHashRowId()); /* * Stage 2: Write JSON Strings to SQL Insert Strings @@ -303,6 +304,7 @@ public static PipelineResult run(Options options) { .withStatementFormatter( new CdcJdbcIO.StatementFormatter() { public String formatStatement(DmlInfo element) { + LOG.debug("Executing SQL: {}", element.getDmlSql()); return element.getDmlSql(); } })); From 33a2b0f2800958df8554620ab43b678253031b03 Mon Sep 17 00:00:00 2001 From: dhercher Date: Tue, 15 Mar 2022 13:16:49 -0700 Subject: [PATCH 079/145] DatastreamToPostgres cleaning up deprecated template (replaced by DatastreamToSQL) PiperOrigin-RevId: 434838199 --- v2/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/v2/pom.xml b/v2/pom.xml index 54b3cc7bd1..651295daba 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -653,7 +653,6 @@ common cdc-parent datastream-to-bigquery - datastream-to-postgres datastream-to-spanner datastream-to-sql elasticsearch-common From 466acc99293c7c22cf2a219d6bfe441631cd15e7 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Mon, 21 Mar 2022 15:41:21 -0700 Subject: [PATCH 080/145] Fix write latency metrics in Pub/Sub to Splunk template. PiperOrigin-RevId: 436323548 --- .../teleport/splunk/SplunkEventWriter.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java b/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java index 9b61f2400e..43a0e446bf 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java +++ b/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java @@ -34,7 +34,6 @@ import java.security.KeyStoreException; import java.security.NoSuchAlgorithmException; import java.security.cert.CertificateException; -import java.time.Instant; import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -272,7 +271,7 @@ private void flush( // Important to close this response to avoid connection leak. response = publisher.execute(events); if (!response.isSuccessStatusCode()) { - UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); int statusCode = response.getStatusCode(); if (statusCode >= 400 && statusCode < 500) { @@ -290,7 +289,7 @@ private void flush( events, response.getStatusMessage(), response.getStatusCode(), receiver); } else { - SUCCESSFUL_WRITE_LATENCY_MS.update(Instant.now().toEpochMilli() - startTime); + SUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); SUCCESS_WRITES.inc(countState.read()); VALID_REQUESTS.inc(); SUCCESSFUL_WRITE_BATCH_SIZE.update(countState.read()); @@ -301,7 +300,7 @@ private void flush( } } catch (HttpResponseException e) { - UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); int statusCode = e.getStatusCode(); if (statusCode >= 400 && statusCode < 500) { @@ -314,7 +313,7 @@ private void flush( flushWriteFailures(events, e.getStatusMessage(), e.getStatusCode(), receiver); } catch (IOException ioe) { - UNSUCCESSFUL_WRITE_LATENCY_MS.update(System.nanoTime() - startTime); + UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); INVALID_REQUESTS.inc(); @@ -414,6 +413,16 @@ private static boolean isValidUrlFormat(String url) { return false; } + /** + * Converts Nanoseconds to Milliseconds. + * + * @param ns time in nanoseconds + * @return time in milliseconds + */ + private static long nanosToMillis(long ns) { + return Math.round(((double) ns) / 1e6); + } + @AutoValue.Builder abstract static class Builder { From 53aef881bb70f06ded91453883e7a338ae5ae28f Mon Sep 17 00:00:00 2001 From: Jeff Date: Tue, 22 Mar 2022 15:42:59 -0400 Subject: [PATCH 081/145] Support Elasticsearch 8.x --- .../teleport/v2/elasticsearch/utils/ElasticsearchIO.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java index eea718c32d..9a764ee6b6 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java @@ -1528,10 +1528,11 @@ static int getBackendVersion(ConnectionConfiguration connectionConfiguration) { (backendVersion == 2 || backendVersion == 5 || backendVersion == 6 - || backendVersion == 7), + || backendVersion == 7, + || backendVersion == 8), "The Elasticsearch version to connect to is %s.x. " + "This version of the ElasticsearchIO is only compatible with " - + "Elasticsearch v7.x, v6.x, v5.x and v2.x", + + "Elasticsearch v8.x, v7.x, v6.x, v5.x and v2.x", backendVersion); return backendVersion; From c33d82dd9b6e14884683d72c08487fad0a474f1e Mon Sep 17 00:00:00 2001 From: Jeff Date: Tue, 22 Mar 2022 15:47:40 -0400 Subject: [PATCH 082/145] Remove unneeded comma from ES versions --- .../cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java index 9a764ee6b6..04278fd396 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java @@ -1528,7 +1528,7 @@ static int getBackendVersion(ConnectionConfiguration connectionConfiguration) { (backendVersion == 2 || backendVersion == 5 || backendVersion == 6 - || backendVersion == 7, + || backendVersion == 7 || backendVersion == 8), "The Elasticsearch version to connect to is %s.x. " + "This version of the ElasticsearchIO is only compatible with " From 1ae5a15149db139ca0ab4fddc68d6b81a3975d16 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 24 Mar 2022 07:50:50 -0700 Subject: [PATCH 083/145] Add option to add pane info to a windowed filename. This was added to Datastream to BigQuery template. Also fix the README for the template. PiperOrigin-RevId: 436993525 --- .../v2/transforms/DLQWriteTransform.java | 15 +- .../v2/transforms/DLQWriteTransformTest.java | 65 +++-- v2/datastream-to-bigquery/README.md | 258 ++++++++++++++---- .../v2/templates/DataStreamToBigQuery.java | 1 + 4 files changed, 262 insertions(+), 77 deletions(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransform.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransform.java index f3fc348761..dee4b6804c 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransform.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransform.java @@ -42,13 +42,15 @@ public class DLQWriteTransform { public abstract static class WriteDLQ extends PTransform, PDone> { public static Builder newBuilder() { - return new AutoValue_DLQWriteTransform_WriteDLQ.Builder(); + return new AutoValue_DLQWriteTransform_WriteDLQ.Builder().setIncludePaneInfo(false); } public abstract String dlqDirectory(); public abstract String tmpDirectory(); + public abstract boolean includePaneInfo(); + @Override public PDone expand(PCollection input) { return input @@ -77,11 +79,16 @@ public void process(ProcessContext context) { .withNumShards(20) .to( new WindowedFilenamePolicy( - dlqDirectory(), "error", "-SSSSS-of-NNNNN", ".json")) + dlqDirectory(), "error", getShardTemplate(), ".json")) .withTempDirectory( FileBasedSink.convertToFileResourceIfPossible(tmpDirectory()))); } + private String getShardTemplate() { + String paneStr = includePaneInfo() ? "-P" : ""; + return paneStr + "-SSSSS-of-NNNNN"; + } + /** Builder for {@link WriteDLQ}. */ @AutoValue.Builder public abstract static class Builder { @@ -101,6 +108,10 @@ public Builder withTmpDirectory(String tmpDirectory) { return setTmpDirectory(tmpDirectory); } + public abstract Builder setIncludePaneInfo(boolean value); + + public abstract boolean includePaneInfo(); + public abstract WriteDLQ build(); } } diff --git a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransformTest.java b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransformTest.java index 6418b31614..3d8624ccfc 100644 --- a/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransformTest.java +++ b/v2/common/src/test/java/com/google/cloud/teleport/v2/transforms/DLQWriteTransformTest.java @@ -20,7 +20,6 @@ import com.google.cloud.teleport.v2.cdc.dlq.FileBasedDeadLetterQueueReconsumer; import java.io.BufferedReader; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileSystems; @@ -30,50 +29,82 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Test the FileBasedDeadLetterQueueReconsumer transform and components. */ @RunWith(JUnit4.class) public class DLQWriteTransformTest { + @Rule public final TestName name = new TestName(); private static final String JSON_ROW_CONTENT = "{\"message\":{\"badcharacters\":\"abc îé def\"}, \"error_message\":\"errorsample3\"}"; - private static final String[] JSON_RESULTS_1 = { - "{\"datasample1\":\"datasample1\",\"_metadata_error\":\"errorsample3\"," - + "\"_metadata_retry_count\":1}", - "{\"datasample2\":\"datasample2\",\"_metadata_error\":\"errorsample3\"," - + "\"_metadata_retry_count\":1}", - "{\"badcharacters\":\"abc îé def\",\"_metadata_error\":\"errorsample3\"," - + "\"_metadata_retry_count\":1}" - }; - - static final Logger LOG = LoggerFactory.getLogger(DLQWriteTransformTest.class); - @Rule public TemporaryFolder folder = new TemporaryFolder(); @Rule public TestPipeline p = TestPipeline.create(); @Test - public void testFilesAreWritten() throws IOException, FileNotFoundException { - File dlqDir = folder.newFolder("dlq/"); + public void testFilesAreWritten() throws IOException { + // Arrange + File dlqDir = folder.newFolder(dlqFolderName(name.getMethodName())); p.apply(Create.of(JSON_ROW_CONTENT).withCoder(StringUtf8Coder.of())) .apply( "Write To DLQ/Writer", DLQWriteTransform.WriteDLQ.newBuilder() .withDlqDirectory(dlqDir.getAbsolutePath()) - .withTmpDirectory(folder.newFolder(".temp/").getAbsolutePath()) + .withTmpDirectory( + folder.newFolder(tmpFolderName(name.getMethodName())).getAbsolutePath()) .build()); + + // Act p.run().waitUntilFinish(); + // Assert File[] files = dlqDir.listFiles(); assertThat(files).isNotEmpty(); ResourceId resourceId = FileSystems.matchNewResource(files[0].getAbsolutePath(), false); + assertThat(resourceId.getFilename()).doesNotContain("pane"); + BufferedReader reader = FileBasedDeadLetterQueueReconsumer.readFile(resourceId); assertThat(reader.readLine()).isEqualTo(JSON_ROW_CONTENT); } + + @Test + public void testFilesAreWrittenWithPaneInfo() throws IOException { + // Arrange + File dlqDir = folder.newFolder(dlqFolderName(name.getMethodName())); + p.apply(Create.of(JSON_ROW_CONTENT).withCoder(StringUtf8Coder.of())) + .apply( + "Write To DLQ/Writer", + DLQWriteTransform.WriteDLQ.newBuilder() + .withDlqDirectory(dlqDir.getAbsolutePath()) + .withTmpDirectory( + folder.newFolder(tmpFolderName(name.getMethodName())).getAbsolutePath()) + .setIncludePaneInfo(true) + .build()); + + // Act + p.run().waitUntilFinish(); + + // Assert + File[] files = dlqDir.listFiles(); + assertThat(files).isNotEmpty(); + + ResourceId resourceId = FileSystems.matchNewResource(files[0].getAbsolutePath(), false); + assertThat(resourceId.getFilename()).contains("pane"); + + BufferedReader reader = FileBasedDeadLetterQueueReconsumer.readFile(resourceId); + assertThat(reader.readLine()).isEqualTo(JSON_ROW_CONTENT); + } + + private static String dlqFolderName(String testName) { + return testName + "/"; + } + + private static String tmpFolderName(String testName) { + return ".temp-" + testName + "/"; + } } diff --git a/v2/datastream-to-bigquery/README.md b/v2/datastream-to-bigquery/README.md index d01be83fc7..4aea2a332d 100644 --- a/v2/datastream-to-bigquery/README.md +++ b/v2/datastream-to-bigquery/README.md @@ -56,74 +56,195 @@ mvn clean package \ ```sh echo '{ "image":"'${TARGET_GCR_IMAGE}'", - "metadata":{"name":"PubSub CDC to BigQuery", - "description":"Replicate Pub/Sub Data into BigQuery Tables", + "metadata":{"name":"Datastream to BigQuery", + "description":"Streaming pipeline. Ingests messages from a stream in Datastream, transforms them, and writes them to a pre-existing BigQuery dataset as a set of tables.", "parameters":[ { - "name":"inputSubscription", - "label":"PubSub Subscription Name", - "helpText":"Full subscription reference", - "paramType":"TEXT" + "label": "File location for Datastream file output in Cloud Storage.", + "help_text": "This is the file location for Datastream file output in Cloud Storage, in the format: gs://${BUCKET}/${ROOT_PATH}/.", + "name": "inputFilePattern", + "param_type": "GCS_READ_FILE" }, { - "name":"autoMapTables", - "label":"Automatically add new BigQuery tables and columns as they appear", - "helpText":"Automatically add new BigQuery tables and columns as they appear", - "paramType":"TEXT", - "isOptional":true + "label": "The Pub/Sub subscription on the Cloud Storage bucket.", + "help_text": "The Pub/Sub subscription used by Cloud Storage to notify Dataflow of new files available for processing, in the format: projects/{PROJECT_NAME}/subscriptions/{SUBSCRIPTION_NAME}", + "name": "gcsPubSubSubscription", + "is_optional": false, + regexes: "^projects\\/[^\\n\\r\\/]+\\/subscriptions\\/[^\\n\\r\\/]+$|^$", + "param_type": "PUBSUB_SUBSCRIPTION" }, { - "name":"outputDatasetTemplate", - "label":"The BigQuery Dataset Name or column template", - "helpText":"The BigQuery Dataset Name or column template", - "paramType":"TEXT", - "isOptional":true + "label": "Datastream output file format (avro/json).", + "help_text": "The format of the output files produced by Datastream. Value can be 'avro' or 'json'.", + "name": "inputFileFormat", + "param_type": "TEXT", + "is_optional": false }, { - "name":"outputTableNameTemplate", - "label":"The BigQuery Table Name or column template", - "helpText":"The BigQuery Table Name or column template", - "paramType":"TEXT", - "isOptional":true + "name": "rfcStartDateTime", + "label": "The starting DateTime used to fetch from GCS (https://tools.ietf.org/html/rfc3339).", + "help_text": "The starting DateTime used to fetch from GCS (https://tools.ietf.org/html/rfc3339).", + "param_type": "TEXT", + "is_optional": true }, { - "name":"outputTableSpec", - "label":"DEPRECATED: Use outputDatasetTemplate AND outputTableNameTemplate", - "helpText":"DEPRECATED: Use outputDatasetTemplate AND outputTableNameTemplate", - "paramType":"TEXT", - "isOptional":true + "name": "javascriptTextTransformGcsPath", + "label": "GCS location of your JavaScript UDF", + "help_text": "The full URL of your .js file. Example: gs://your-bucket/your-function.js", + regexes: "^gs:\\/\\/[^\\n\\r]+$", + "param_type": "GCS_READ_FILE", + "is_optional": true }, - {"name":"outputDeadletterTable", - "label":"Deadletter Queue Table", - "helpText":"DLQ Table Ref: PROJECT:dataset.dlq", - "paramType":"TEXT" + { + "name": "javascriptTextTransformFunctionName", + "label": "The name of the JavaScript function you wish to call as your UDF", + "help_text": "The function name should only contain letters, digits and underscores. Example: 'transform' or 'transform_udf1'.", + regexes: "[a-zA-Z0-9_]+", + "param_type": "TEXT", + "is_optional": true }, { - "name":"autoscalingAlgorithm","label":"Autoscaling algorithm to use", - "helpText":"Autoscaling algorithm to use: THROUGHPUT_BASED", - "paramType":"TEXT", - "isOptional":true + "name": "outputStagingDatasetTemplate", + "label": "Name or template for the dataset to contain staging tables.", + "help_text": "This is the name for the dataset to contain staging tables. This parameter supports templates (e.g. {_metadata_dataset}_log or my_dataset_log). Normally, this parameter is a dataset name.", + "param_type": "TEXT" }, { - "name":"numWorkers","label":"Number of workers Dataflow will start with", - "helpText":"Number of workers Dataflow will start with", - "paramType":"TEXT", - "isOptional":true + "name": "outputDatasetTemplate", + "label": "Template for the dataset to contain replica tables.", + "help_text": "This is the name for the dataset to contain replica tables. This parameter supports templates (e.g. {_metadata_dataset} or my_dataset). Normally, this parameter is a dataset name.", + "param_type": "TEXT" + }, + { + "label": "Project name for BigQuery datasets.", + "help_text": "Project for BigQuery datasets to output data into. The default for this parameter is the project where the Dataflow pipeline is running.", + "name": "outputProjectId", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Template for the name of staging tables.", + "name": "outputStagingTableNameTemplate", + "help_text": "This is the template for the name of staging tables (e.g. {_metadata_table}). Default is {_metadata_table}.", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Template for the name of replica tables.", + "name": "outputTableNameTemplate", + "help_text": "This is the template for the name of replica tables (e.g. {_metadata_table}). Default is {_metadata_table}.", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Dead letter queue directory.", + "name": "deadLetterQueueDirectory", + "help_text": "This is the file path for Dataflow to write the dead letter queue output. This path should not be in the same path as the Datastream file output.", + "is_optional": false, + "param_type": "TEXT" + }, + { + "label": "Name or template for the stream to poll for schema information.", + "name": "streamName", + "help_text": "This is the name or template for the stream to poll for schema information. Default is {_metadata_stream}. The default value is enough under most conditions.", + "is_optional": true, + "param_type": "TEXT" + }, + { + "name":"dataStreamRootUrl", + "label":"Datastream API URL (only required for testing)", + "help_text": "Datastream API URL", + "param_type": "TEXT", + "is_optional": true + }, + { + "name": "mergeFrequencyMinutes", + "label": "The number of minutes between merges for a given table.", + "help_text": "The number of minutes between merges for a given table.", + "param_type": "TEXT", + "is_optional": true + }, + { + "name": "dlqRetryMinutes", + "label": "The number of minutes between DLQ Retries.", + "help_text": "The number of minutes between DLQ Retries.", + "param_type": "TEXT", + "is_optional": true }, - { - "name":"maxNumWorkers","label":"Maximum number of workers Dataflow job will use", - "helpText":"Maximum number of workers Dataflow job will use", - "paramType":"TEXT", - "isOptional":true + "name": "applyMerge", + "label": "A switch to disable MERGE queries for the job.", + "help_text": "A switch to disable MERGE queries for the job.", + "param_type": "TEXT", + "is_optional": true }, { - "name":"workerMachineType","label":"Worker Machine Type to use in Dataflow Job", - "helpText":"Machine Type to Use: n1-standard-4", - "paramType":"TEXT", - "isOptional":true + "name":"autoscalingAlgorithm", + "label":"Autoscaling algorithm to use", + "help_text": "Autoscaling algorithm to use: THROUGHPUT_BASED", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"numWorkers", + "label":"Number of workers Dataflow will start with", + "help_text": "Number of workers Dataflow will start with", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"maxNumWorkers", + "label":"Maximum number of workers Dataflow job will use", + "help_text": "Maximum number of workers Dataflow job will use", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"numberOfWorkerHarnessThreads", + "label":"Dataflow job will use max number of threads per worker", + "help_text": "Maximum number of threads per worker Dataflow job will use", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"dumpHeapOnOOM", + "label":"Dataflow will dump heap on an OOM error", + "help_text": "Dataflow will dump heap on an OOM error", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"saveHeapDumpsToGcsPath", + "label":"Dataflow will dump heap on an OOM error to supplied GCS path", + "help_text": "Dataflow will dump heap on an OOM error to supplied GCS path", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"workerMachineType", + "label":"Worker Machine Type to use in Dataflow Job", + "help_text": "Machine Type to Use: n1-standard-4", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"maxStreamingRowsToBatch", + "label":"Max number of rows per BigQueryIO batch", + "help_text": "Max number of rows per BigQueryIO batch", + "param_type": "TEXT", + "is_optional": true + }, + { + "name":"maxStreamingBatchSize", + "label":"Maximum byte size of a single streaming insert to BigQuery.", + "help_text":"Sets the maximum byte size of a single streaming insert to BigQuery. This option could fix 'row too large' errors.", + "param_type":"TEXT", + "regexes":[ + "^[1-9][0-9]+$", + ], + "is_optional":true } - ]}, + ]}, "sdk_info":{"language":"JAVA"} }' > image_spec.json gsutil cp image_spec.json ${TEMPLATE_IMAGE_SPEC} @@ -140,17 +261,33 @@ mvn test ### Executing Template The template requires the following parameters: -* inputSubscription: PubSub subscription to read from (ie. projects//subscriptions/) -* outputDatasetTemplate: The name of the dataset or templated logic to extract it (ie. 'prefix_{schema_name}') -* outputTableNameTemplate: The name of the table or templated logic to extract it (ie. 'prefix_{table_name}') -* outputDeadletterTable: Deadletter table for failed inserts in form: project-id:dataset.table +* inputFilePattern: This is the file location for Datastream file output in Cloud Storage, in the format: gs://${BUCKET}/${ROOT_PATH}/. +* gcsPubSubSubscription: The Pub/Sub subscription used by Cloud Storage to notify Dataflow of new files available for processing, in the format: projects/{PROJECT_NAME}/subscriptions/{SUBSCRIPTION_NAME} +* inputFileFormat: The format of the output files produced by Datastream. Value can be 'avro' or 'json'. +* outputStagingDatasetTemplate: This is the name for the dataset to contain staging tables. This parameter supports templates (e.g. {_metadata_dataset}_log or my_dataset_log). Normally, this parameter is a dataset name. +* outputDatasetTemplate: This is the name for the dataset to contain replica tables. This parameter supports templates (e.g. {_metadata_dataset} or my_dataset). Normally, this parameter is a dataset name. +* deadLetterQueueDirectory: This is the file path for Dataflow to write the dead letter queue output. This path should not be in the same path as the Datastream file output. The template has the following optional parameters: -* javascriptTextTransformGcsPath: Gcs path to javascript udf source. Udf will be preferred option for transformation if supplied. Default: null -* javascriptTextTransformFunctionName: UDF Javascript Function Name. Default: null - -* maxRetryAttempts: Max retry attempts, must be > 0. Default: no retries -* maxRetryDuration: Max retry duration in milliseconds, must be > 0. Default: no retries +* rfcStartDateTime: The starting DateTime used to fetch from GCS (https://tools.ietf.org/html/rfc3339). +* javascriptTextTransformGcsPath: The full URL of your .js file. Example: gs://your-bucket/your-function.js +* javascriptTextTransformFunctionName: The function name should only contain letters, digits and underscores. Example: 'transform' or 'transform_udf1'. +* outputProjectId: Project for BigQuery datasets to output data into. The default for this parameter is the project where the Dataflow pipeline is running. +* outputStagingTableNameTemplate: This is the template for the name of staging tables (e.g. {_metadata_table}). Default is {_metadata_table}. +* outputTableNameTemplate: This is the template for the name of replica tables (e.g. {_metadata_table}). Default is {_metadata_table}. +* streamName: This is the name or template for the stream to poll for schema information. Default is {_metadata_stream}. The default value is enough under most conditions. +* dataStreamRootUrl: Datastream API URL (only required for testing) +* mergeFrequencyMinutes: The number of minutes between merges for a given table. +* dlqRetryMinutes: The number of minutes between DLQ Retries. +* applyMerge: A switch to disable MERGE queries for the job. +* autoscalingAlgorithm: Autoscaling algorithm to use: THROUGHPUT_BASED +* numWorkers: Number of workers Dataflow will start with +* maxNumWorkers: Maximum number of workers Dataflow job will use +* numberOfWorkerHarnessThreads: Maximum number of threads per worker Dataflow job will use +* dumpHeapOnOOM: Dataflow will dump heap on an OOM error +* saveHeapDumpsToGcsPath: Dataflow will dump heap on an OOM error to supplied GCS path +* workerMachineType: Machine Type to Use: n1-standard-4 +* maxStreamingRowsToBatch: Max number of rows per BigQueryIO batch Template can be executed using the following API call: ```sh @@ -158,5 +295,10 @@ export JOB_NAME="${IMAGE_NAME}-`date +%Y%m%d-%H%M%S-%N`" gcloud beta dataflow flex-template run ${JOB_NAME} \ --project=${PROJECT} --region=us-central1 \ --template-file-gcs-location=${TEMPLATE_IMAGE_SPEC} \ - --parameters inputSubscription=${SUBSCRIPTION},outputDeadletterTable=${DEADLETTER_TABLE} + --parameters=inputFilePattern=${FILE_PATTERN} \ + --parameters=gcsPubSubSubscription=${GCS_PUBSUB_SUBSCRIPTION} \ + --parameters=inputFileFormat=${INPUT_FILE_FORMAT} \ + --parameters=outputStagingDatasetTemplate=${OUTPUT_STAGING_DATASET_TEMPLATE} \ + --parameters=outputDatasetTemplate=${OUTPUT_DATASET_TEMPLATE} \ + --parameters=deadLetterQueueDirectory=${DLQ_DIR} ``` diff --git a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java index 024376ab7e..8e3f03f69c 100644 --- a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java +++ b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java @@ -444,6 +444,7 @@ public void process( DLQWriteTransform.WriteDLQ.newBuilder() .withDlqDirectory(dlqDirectory) .withTmpDirectory(tempDlqDir) + .setIncludePaneInfo(true) .build()); // Execute the pipeline and return the result. From 9d0ef8206e692616cc735dbed92387b62fcf3ac0 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Thu, 24 Mar 2022 08:42:31 -0700 Subject: [PATCH 084/145] Upgrade to Beam 2.37 PiperOrigin-RevId: 437003186 --- pom.xml | 2 +- v2/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 546274597c..6cca8e6d0f 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ 1.0-rc6 1.8.2 8.7 - 2.36.0 + 2.37.0 1.3 2.1 1.8 diff --git a/v2/pom.xml b/v2/pom.xml index 651295daba..4851bfb8f5 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -29,7 +29,7 @@ 1.7.4 1.0-rc6 8.7 - 2.36.0 + 2.37.0 2.1 2.10.1 30.1-jre From 6322c59c186ff8d94812a604af303f0ed999cf85 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Thu, 24 Mar 2022 17:21:36 -0700 Subject: [PATCH 085/145] Add basic transforms to convert data change records into Text/Avro and write them into a GCS output directory. PiperOrigin-RevId: 437124580 --- .../WriteDataChangeRecordsToGcsAvro.java | 154 ++++++++++++++++ .../WriteDataChangeRecordsToGcsText.java | 172 ++++++++++++++++++ .../WriteDataChangeRecordsToGcsAvroTest.java | 158 ++++++++++++++++ .../WriteDataChangeRecordsToGcsTextTest.java | 160 ++++++++++++++++ 4 files changed, 644 insertions(+) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java new file mode 100644 index 0000000000..570d7ed9e0 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java @@ -0,0 +1,154 @@ +/* + * Copyright (C) 2022 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.v2.transforms; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; +import com.google.cloud.teleport.v2.io.WindowedFilenamePolicy; +import com.google.cloud.teleport.v2.utils.WriteToGCSUtility; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +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.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link WriteDataChangeRecordToGcsAvro} class is a {@link PTransform} that takes in {@link + * PCollection} of Spanner data change records. The transform converts and writes these records to + * GCS in avro file format. + */ +@AutoValue +public abstract class WriteDataChangeRecordsToGcsAvro + extends PTransform, PDone> { + @VisibleForTesting protected static final String DEFAULT_OUTPUT_FILE_PREFIX = "output"; + /* Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(WriteDataChangeRecordsToGcsAvro.class); + + public static WriteToGcsBuilder newBuilder() { + return new AutoValue_WriteDataChangeRecordsToGcsAvro.Builder(); + } + + public abstract String outputDirectory(); + + public abstract String outputFilenamePrefix(); + + public abstract String tempLocation(); + + public abstract Integer numShards(); + + @Override + public PDone expand(PCollection dataChangeRecords) { + return dataChangeRecords + /* + * Writing as avro file using {@link AvroIO}. + * + * The {@link WindowedFilenamePolicy} class specifies the file path for writing the file. + * The {@link withNumShards} option specifies the number of shards passed by the user. + * The {@link withTempDirectory} option sets the base directory used to generate temporary files. + */ + .apply( + "Writing as Avro", + AvroIO.write(DataChangeRecord.class) + .to( + new WindowedFilenamePolicy( + outputDirectory(), + outputFilenamePrefix(), + WriteToGCSUtility.SHARD_TEMPLATE, + WriteToGCSUtility.FILE_SUFFIX_MAP.get(WriteToGCSUtility.FileFormat.AVRO))) + .withTempDirectory( + FileBasedSink.convertToFileResourceIfPossible(tempLocation()).getCurrentDirectory()) + .withWindowedWrites() + .withNumShards(numShards())); + } + + /** + * The {@link WriteToGcsAvroOptions} interface provides the custom execution options passed by the + * executor at the command-line. + */ + public interface WriteToGcsAvroOptions extends PipelineOptions { + @Description("The directory to output files to. Must end with a slash.") + String getOutputDirectory(); + + void setOutputDirectory(String outputDirectory); + + @Description( + "The filename prefix of the files to write to. Default file prefix is set to \"output\".") + @Default.String("output") + String getOutputFilenamePrefix(); + + void setOutputFilenamePrefix(String outputFilenamePrefix); + + @Description( + "The maximum number of output shards produced when writing. Default number is runner" + + " defined.") + @Default.Integer(1) + Integer getNumShards(); + + void setNumShards(Integer numShards); + } + + /** Builder for {@link WriteDataChangeRecordsToGcsAvro}. */ + @AutoValue.Builder + public abstract static class WriteToGcsBuilder { + abstract WriteToGcsBuilder setOutputDirectory(String outputDirectory); + + abstract String outputDirectory(); + + abstract WriteToGcsBuilder setTempLocation(String tempLocation); + + abstract String tempLocation(); + + abstract WriteToGcsBuilder setOutputFilenamePrefix(String outputFilenamePrefix); + + abstract WriteToGcsBuilder setNumShards(Integer numShards); + + abstract WriteDataChangeRecordsToGcsAvro autoBuild(); + + public WriteToGcsBuilder withOutputDirectory(String outputDirectory) { + checkArgument( + outputDirectory != null, "withOutputDirectory(outputDirectory) called with null input."); + return setOutputDirectory(outputDirectory); + } + + public WriteToGcsBuilder withTempLocation(String tempLocation) { + checkArgument(tempLocation != null, "withTempLocation(tempLocation) called with null input."); + return setTempLocation(tempLocation); + } + + public WriteToGcsBuilder withOutputFilenamePrefix(String outputFilenamePrefix) { + if (outputFilenamePrefix == null) { + LOG.info("Defaulting output filename prefix to: {}", DEFAULT_OUTPUT_FILE_PREFIX); + outputFilenamePrefix = DEFAULT_OUTPUT_FILE_PREFIX; + } + return setOutputFilenamePrefix(outputFilenamePrefix); + } + + public WriteDataChangeRecordsToGcsAvro build() { + checkNotNull(outputDirectory(), "Provide output directory to write to. "); + checkNotNull(tempLocation(), "Temporary directory needs to be provided. "); + return autoBuild(); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java new file mode 100644 index 0000000000..79b1e89c35 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java @@ -0,0 +1,172 @@ +/* + * Copyright (C) 2022 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.v2.transforms; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; +import com.google.cloud.teleport.v2.io.WindowedFilenamePolicy; +import com.google.cloud.teleport.v2.utils.WriteToGCSUtility; +import com.google.gson.Gson; +import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +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.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link WriteDataChangeRecordsToGcsText} class is a {@link PTransform} that takes in {@link + * PCollection} of Spanner data change records. The transform converts and writes these records to + * GCS in JSON text file format. + */ +@AutoValue +public abstract class WriteDataChangeRecordsToGcsText + extends PTransform, PDone> { + @VisibleForTesting protected static final String DEFAULT_OUTPUT_FILE_PREFIX = "output"; + /* Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(WriteDataChangeRecordsToGcsText.class); + + public static WriteToGcsBuilder newBuilder() { + return new AutoValue_WriteDataChangeRecordsToGcsText.Builder(); + } + + public abstract String outputDirectory(); + + public abstract String outputFilenamePrefix(); + + public abstract String tempLocation(); + + public abstract Integer numShards(); + + @Override + public PDone expand(PCollection dataChangeRecords) { + return dataChangeRecords + /* + * Converting DataChangeRecords to JSON text using DoFn and {@link + * DataChangeRecordToJsonTextFn} class. + */ + .apply("Transform to JSON Text", MapElements.via(new DataChangeRecordToJsonTextFn())) + /* + * Writing as text file using {@link TextIO}. + * + * The {@link WindowedFilenamePolicy} class specifies the file path for writing the file. + * The {@link withNumShards} option specifies the number of shards passed by the user. + * The {@link withTempDirectory} option sets the base directory used to generate temporary files. + */ + .apply( + "Writing as Text", + TextIO.write() + .to( + new WindowedFilenamePolicy( + outputDirectory(), + outputFilenamePrefix(), + WriteToGCSUtility.SHARD_TEMPLATE, + WriteToGCSUtility.FILE_SUFFIX_MAP.get(WriteToGCSUtility.FileFormat.TEXT))) + .withTempDirectory( + FileBasedSink.convertToFileResourceIfPossible(tempLocation()) + .getCurrentDirectory()) + .withWindowedWrites() + .withNumShards(numShards())); + } + + static class DataChangeRecordToJsonTextFn extends SimpleFunction { + private static Gson gson = new Gson(); + + @Override + public String apply(DataChangeRecord record) { + return gson.toJson(record, DataChangeRecord.class); + } + } + + /** + * The {@link WriteToGcsTextOptions} interface provides the custom execution options passed by the + * executor at the command-line. + */ + public interface WriteToGcsTextOptions extends PipelineOptions { + @Description("The directory to output files to. Must end with a slash.") + String getOutputDirectory(); + + void setOutputDirectory(String outputDirectory); + + @Description( + "The filename prefix of the files to write to. Default file prefix is set to \"output\". ") + @Default.String("output") + String getOutputFilenamePrefix(); + + void setOutputFilenamePrefix(String outputFilenamePrefix); + + @Description( + "The maximum number of output shards produced when writing. Default number is runner" + + " defined.") + @Default.Integer(1) + Integer getNumShards(); + + void setNumShards(Integer numShards); + } + + /** Builder for {@link WriteDataChangeRecordsToGcsText}. */ + @AutoValue.Builder + public abstract static class WriteToGcsBuilder { + abstract WriteToGcsBuilder setOutputDirectory(String outputDirectory); + + abstract String outputDirectory(); + + abstract WriteToGcsBuilder setTempLocation(String tempLocation); + + abstract String tempLocation(); + + abstract WriteToGcsBuilder setOutputFilenamePrefix(String outputFilenamePrefix); + + abstract WriteToGcsBuilder setNumShards(Integer numShards); + + abstract WriteDataChangeRecordsToGcsText autoBuild(); + + public WriteToGcsBuilder withOutputDirectory(String outputDirectory) { + checkArgument( + outputDirectory != null, "withOutputDirectory(outputDirectory) called with null input."); + return setOutputDirectory(outputDirectory); + } + + public WriteToGcsBuilder withTempLocation(String tempLocation) { + checkArgument(tempLocation != null, "withTempLocation(tempLocation) called with null input."); + return setTempLocation(tempLocation); + } + + public WriteToGcsBuilder withOutputFilenamePrefix(String outputFilenamePrefix) { + if (outputFilenamePrefix == null) { + LOG.info("Defaulting output filename prefix to: {}", DEFAULT_OUTPUT_FILE_PREFIX); + outputFilenamePrefix = DEFAULT_OUTPUT_FILE_PREFIX; + } + return setOutputFilenamePrefix(outputFilenamePrefix); + } + + public WriteDataChangeRecordsToGcsText build() { + checkNotNull(outputDirectory(), "Provide output directory to write to."); + checkNotNull(tempLocation(), "Temporary directory needs to be provided."); + return autoBuild(); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java new file mode 100644 index 0000000000..326c7fe54a --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java @@ -0,0 +1,158 @@ +/* + * Copyright (C) 2022 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.v2.transforms; + +import com.google.cloud.Timestamp; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ColumnType; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ModType; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.TypeCode; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ValueCaptureType; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test cases for the {@link WriteDataChangeRecordsToGcsAvro} class. */ +@RunWith(JUnit4.class) +public class WriteDataChangeRecordsToGcsAvroTest { + /** Rule for pipeline testing. */ + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + /** Rule for exception testing. */ + @Rule public ExpectedException expectedException = ExpectedException.none(); + /** Rule for temporary folder storing output records. */ + @Rule public final TemporaryFolder tmpDir = new TemporaryFolder(); + + private static final String AVRO_FILENAME_PREFIX = "avro-output-"; + private static final Integer NUM_SHARDS = 1; + private static String fakeDir; + private static String fakeTempLocation; + private PipelineOptions options; + + @Before + public void setUp() throws InterruptedException, IOException { + options = TestPipeline.testingPipelineOptions(); + fakeDir = tmpDir.newFolder("output").getAbsolutePath(); + fakeTempLocation = tmpDir.newFolder("temporaryLocation").getAbsolutePath(); + } + + /** Test the basic WriteDataChangeRecordsToGcsAvro transform. */ + @Test + public void testBasicWrite() { + // First run the transform in a separate pipeline. + final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); + Pipeline p = Pipeline.create(options); + p.apply("CreateInput", Create.of(dataChangeRecord)) + .apply( + "WriteTextFile(s)", + WriteDataChangeRecordsToGcsAvro.newBuilder() + .withOutputDirectory(fakeDir) + .withOutputFilenamePrefix(AVRO_FILENAME_PREFIX) + .setNumShards(NUM_SHARDS) + .withTempLocation(fakeTempLocation) + .build()); + p.run(); + + // Then, read the records back from the output directory using AvrioIO.read. + PCollection dataChangeRecords = + pipeline.apply( + "readRecords", + AvroIO.read(DataChangeRecord.class) + .from(fakeDir + "/avro-output-GlobalWindow-pane-0-last-00-of-01.avro")); + PAssert.that(dataChangeRecords).containsInAnyOrder(dataChangeRecord); + pipeline.run(); + } + + /** + * Test whether {@link WriteDataChangeRecordsToGcsAvro} throws an exception if no output directory + * is provided. + */ + @Test + public void testWriteWithoutOutputDirectory() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("withOutputDirectory(outputDirectory) called with null input."); + final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); + pipeline + .apply("CreateInput", Create.of(dataChangeRecord)) + .apply( + "WriteTextFile(s)", + WriteDataChangeRecordsToGcsAvro.newBuilder() + .withOutputDirectory(null) + .withOutputFilenamePrefix(AVRO_FILENAME_PREFIX) + .setNumShards(NUM_SHARDS) + .withTempLocation(fakeTempLocation) + .build()); + pipeline.run(); + } + /** + * Test whether {@link WriteDataChangeRecordsToGcsAvro} throws an exception if temporary directory + * is not provided. + */ + @Test + public void testWriteWithoutTempLocation() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("withTempLocation(tempLocation) called with null input."); + final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); + pipeline + .apply("CreateInput", Create.of(dataChangeRecord)) + .apply( + "WriteTextFile(s)", + WriteDataChangeRecordsToGcsAvro.newBuilder() + .withOutputDirectory(fakeDir) + .withOutputFilenamePrefix(AVRO_FILENAME_PREFIX) + .setNumShards(NUM_SHARDS) + .withTempLocation(null) + .build()); + pipeline.run(); + } + + private DataChangeRecord createTestDataChangeRecord() { + return new DataChangeRecord( + "partitionToken", + Timestamp.ofTimeSecondsAndNanos(10L, 20), + "serverTransactionId", + true, + "1", + "tableName", + Arrays.asList( + new ColumnType("column1", new TypeCode("type1"), true, 1L), + new ColumnType("column2", new TypeCode("type2"), false, 2L)), + Collections.singletonList( + new Mod( + "{\"column1\": \"value1\"}", + "{\"column2\": \"oldValue2\"}", + "{\"column2\": \"newValue2\"}")), + ModType.UPDATE, + ValueCaptureType.OLD_AND_NEW_VALUES, + 10L, + 2L, + null); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java new file mode 100644 index 0000000000..b5614a7cfa --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java @@ -0,0 +1,160 @@ +/* + * Copyright (C) 2022 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.v2.transforms; + +import com.google.cloud.Timestamp; +import com.google.gson.Gson; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ColumnType; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ModType; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.TypeCode; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ValueCaptureType; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test cases for the {@link WriteDataChangeRecordsToGcsText} class. */ +@RunWith(JUnit4.class) +public class WriteDataChangeRecordsToGcsTextTest { + /** Rule for pipeline testing. */ + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + /** Rule for exception testing. */ + @Rule public ExpectedException expectedException = ExpectedException.none(); + /** Rule for temporary folder for output records. */ + @Rule public final TemporaryFolder tmpDir = new TemporaryFolder(); + + private static final String TEXT_FILENAME_PREFIX = "text-output-"; + private static final Integer NUM_SHARDS = 1; + private static String fakeDir; + private static String fakeTempLocation; + private PipelineOptions options; + private final Gson gson = new Gson(); + + @Before + public void setUp() throws InterruptedException, IOException { + options = TestPipeline.testingPipelineOptions(); + fakeDir = tmpDir.newFolder("output").getAbsolutePath(); + fakeTempLocation = tmpDir.newFolder("temporaryLocation").getAbsolutePath(); + } + + /** Test the basic WriteDataChangeRecordsToGcsText transform. */ + @Test + public void testBasicWrite() { + // First run the transform in a separate pipeline. + final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); + Pipeline p = Pipeline.create(options); + p.apply("CreateInput", Create.of(dataChangeRecord)) + .apply( + "WriteTextFile(s)", + WriteDataChangeRecordsToGcsText.newBuilder() + .withOutputDirectory(fakeDir) + .withOutputFilenamePrefix(TEXT_FILENAME_PREFIX) + .setNumShards(NUM_SHARDS) + .withTempLocation(fakeTempLocation) + .build()); + p.run(); + + // Then, read the records back from the output directory using TextIO.read. + PCollection dataChangeRecords = + pipeline.apply( + "readRecords", + TextIO.read().from(fakeDir + "/text-output-GlobalWindow-pane-0-last-00-of-01.txt")); + PAssert.that(dataChangeRecords) + .containsInAnyOrder(gson.toJson(dataChangeRecord, DataChangeRecord.class)); + pipeline.run(); + } + + /** + * Test whether {@link WriteDataChangeRecordsToGcsText} throws an exception if no output directory + * is provided. + */ + @Test + public void testWriteWithoutOutputDirectory() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("withOutputDirectory(outputDirectory) called with null input."); + final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); + pipeline + .apply("CreateInput", Create.of(dataChangeRecord)) + .apply( + "WriteTextFile(s)", + WriteDataChangeRecordsToGcsText.newBuilder() + .withOutputDirectory(null) + .withOutputFilenamePrefix(TEXT_FILENAME_PREFIX) + .setNumShards(NUM_SHARDS) + .withTempLocation(fakeTempLocation) + .build()); + pipeline.run(); + } + /** + * Test whether {@link WriteDataChangeRecordsToGcsText} throws an exception if temporary directory + * is not provided. + */ + @Test + public void testWriteWithoutTempLocation() { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("withTempLocation(tempLocation) called with null input."); + final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); + pipeline + .apply("CreateInput", Create.of(dataChangeRecord)) + .apply( + "WriteTextFile(s)", + WriteDataChangeRecordsToGcsText.newBuilder() + .withOutputDirectory(fakeDir) + .withOutputFilenamePrefix(TEXT_FILENAME_PREFIX) + .setNumShards(NUM_SHARDS) + .withTempLocation(null) + .build()); + pipeline.run(); + } + + private DataChangeRecord createTestDataChangeRecord() { + return new DataChangeRecord( + "partitionToken", + Timestamp.ofTimeSecondsAndNanos(10L, 20), + "serverTransactionId", + true, + "1", + "tableName", + Arrays.asList( + new ColumnType("column1", new TypeCode("type1"), true, 1L), + new ColumnType("column2", new TypeCode("type2"), false, 2L)), + Collections.singletonList( + new Mod( + "{\"column1\": \"value1\"}", + "{\"column2\": \"oldValue2\"}", + "{\"column2\": \"newValue2\"}")), + ModType.UPDATE, + ValueCaptureType.OLD_AND_NEW_VALUES, + 10L, + 2L, + null); + } +} From 601d3d02816a6a301785e7ee73e90ebe4fc9d8cf Mon Sep 17 00:00:00 2001 From: olegsa Date: Fri, 25 Mar 2022 14:53:33 -0700 Subject: [PATCH 086/145] Fix testAssetWithEntityJsonToParquetFailOnExistingFilesE2E The CL updates the test to make it independent from the order of the input files. The test pipelines are required to be run if the transforms are applied to them by the testing framework. PiperOrigin-RevId: 437340768 --- .../v2/templates/DataplexFileFormatConversionTest.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java index 5308fcb28e..c3c3c82623 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexFileFormatConversionTest.java @@ -310,7 +310,7 @@ public void testAssetWithEntityJsonToParquetSkipExistingFilesE2E() throws IOExce options.setOutputAsset(outputAsset.getName()); options.setWriteDisposition(WriteDispositionOptions.SKIP); - // simulate the file 1.json -> 1.parquet already + // simulate the 1.json -> 1.parquet conversion already happened copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/1.parquet"); // run the pipeline, only 2.json -> 2.parquet conversion should happen @@ -359,7 +359,7 @@ public void testAssetWithEntityJsonToParquetFailOnExistingFilesE2E() throws IOEx when(dataplex.getPartitions(entity2.getName())).thenReturn(ImmutableList.of()); when(dataplex.getAsset(outputAsset.getName())).thenReturn(outputAsset); - // setup options to skip existing files + // setup options to fail on existing files FileFormatConversionOptions options = PipelineOptionsFactory.create().as(FileFormatConversionOptions.class); options.setInputAssetOrEntitiesList(asset2.getName()); @@ -367,10 +367,12 @@ public void testAssetWithEntityJsonToParquetFailOnExistingFilesE2E() throws IOEx options.setOutputAsset(outputAsset.getName()); options.setWriteDisposition(WriteDispositionOptions.FAIL); - // simulate the file 1.json -> 1.parquet already + // simulate the 1.json -> 1.parquet conversion already happened copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/1.parquet"); + // simulate the 2.json -> 2.parquet conversion already happened + copyFileToOutputBucket("entity2.existing/1.parquet", "entity2/2.parquet"); - // run the pipeline, only 2.json -> 2.parquet conversion should happen + // run the pipeline, the job should fail because 1.parquet already exists DataplexFileFormatConversion.run( mainPipeline, options, dataplex, DataplexFileFormatConversionTest::outputPathProvider) .waitUntilFinish(); From 9a72151049b29265a9163147275e1c76df69fd52 Mon Sep 17 00:00:00 2001 From: dhercher Date: Sun, 27 Mar 2022 05:55:22 -0700 Subject: [PATCH 087/145] DatastreamToSQL: Add failing SQL statement to error message logged PiperOrigin-RevId: 437576372 --- .../main/java/com/google/cloud/teleport/v2/io/CdcJdbcIO.java | 5 ++++- .../com/google/cloud/teleport/v2/utils/DatastreamToDML.java | 4 ++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/io/CdcJdbcIO.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/io/CdcJdbcIO.java index 78058aa150..5e832a4e63 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/io/CdcJdbcIO.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/io/CdcJdbcIO.java @@ -589,7 +589,10 @@ private void executeBatchSingleStatementFormatting() statement.executeUpdate(formattedStatement); connection.commit(); } catch (SQLException exception) { - LOG.error("SQLException Occurred: {}", exception.toString()); + LOG.error( + "SQLException Occurred: {} while executing statement: {}", + exception.toString(), + formattedStatement); connection.rollback(); } } diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java index ab14282c32..b6b56c1750 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java @@ -106,7 +106,10 @@ public void processElement(ProcessContext context) { // Null rows suggest no DML is required. if (dmlInfo != null) { + LOG.debug("Output Data: {}", jsonString); context.output(KV.of(dmlInfo.getStateWindowKey(), dmlInfo)); + } else { + LOG.debug("Skipping Null DmlInfo: {}", jsonString); } } catch (IOException e) { // TODO(dhercher): Push failure to DLQ collection @@ -209,6 +212,7 @@ public DmlInfo convertJsonToDmlInfo(JsonNode rowObj, String failsafeValue) { Map tableSchema = this.getTableSchema(catalogName, schemaName, tableName); if (tableSchema.isEmpty()) { // If the table DNE we return null (NOOP) + LOG.debug("Table Not Found: {}.{}.{}", catalogName, schemaName, tableName); return null; } From 86de88bad6c99763a2a9b4155c16b835f55721e5 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Sat, 12 Feb 2022 15:02:55 -0500 Subject: [PATCH 088/145] Add the basics for executing a build --- cicd/cmd/run-build/main.go | 106 +++++++++++++++++++ cicd/internal/op/maven.go | 10 +- cicd/internal/op/run-cmd.go | 23 +++- cicd/internal/repo/files.go | 33 +++--- cicd/internal/repo/modules.go | 163 +++++++++++++---------------- cicd/internal/repo/modules_test.go | 10 +- 6 files changed, 228 insertions(+), 117 deletions(-) create mode 100644 cicd/cmd/run-build/main.go diff --git a/cicd/cmd/run-build/main.go b/cicd/cmd/run-build/main.go new file mode 100644 index 0000000000..b70dc65213 --- /dev/null +++ b/cicd/cmd/run-build/main.go @@ -0,0 +1,106 @@ +/* + * Copyright (C) 2022 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 main + +import ( + "flag" + "log" + "strings" + + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/flags" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" +) + +const ( + BuildCommand = "clean install" + FlexPom = "v2/pom.xml" +) + +var ( + args = []string{ + "-am", // Include anything this depends on for more reliable builds + "-amd", // Include anything that depends on this to guarantee no breaking changes + "-Dmaven.test.skip", // Skip tests for verifying this builds + "-Dmdep.analyze.skip", // Skip analyzing for dependency failings + "-Djib.skip", // Skip Jib, because don't care about images + } +) + +func main() { + flags.RegisterCommonFlags() + flag.Parse() + + changed := flags.ChangedFiles() + if len(changed) == 0 { + return + } + + for root, children := range repo.GetModulesForPaths(changed) { + var err error + if root == repo.ClassicRoot { + err = op.RunMavenOnPom(root, BuildCommand, args...) + } else if root == repo.FlexRoot { + // A change to the root POM could impact all Flex Templates, so we must rebuild everything on + // those changes. + if buildAllFlexTemplates(changed) { + log.Println("A change indicated to build all Flex Modules. This may take multiple minutes.") + err = op.RunMavenOnPom(root, BuildCommand, args...) + } else { + children = removeRoot(children) + if len(children) != 0 { + err = op.RunMavenOnModule(root, BuildCommand, strings.Join(children, ","), args...) + } else { + log.Println("The only Flex changes were to files that should not trigger a build") + } + } + } + + if err != nil { + log.Fatalf("%v", err) + } + } +} + +// Returns true if all flex templates need to be built. Otherwise, false. +func buildAllFlexTemplates(changed []string) bool { + for _, c := range changed { + if c == FlexPom { + return true + } + } + return false +} + +// Removes root and returns results. This may reorder the input. +func removeRoot(flexModules []string) []string { + var i int + for i = 0; i < len(flexModules); i += 1 { + if flexModules[i] == "" { + break + } + } + + if i == len(flexModules) { + return flexModules + } + + // Order doesn't matter when passing the modules + l := len(flexModules) + flexModules[i] = flexModules[l - 1] + return flexModules[:l - 1] +} diff --git a/cicd/internal/op/maven.go b/cicd/internal/op/maven.go index 4093b6c1b1..09e1f1d8af 100644 --- a/cicd/internal/op/maven.go +++ b/cicd/internal/op/maven.go @@ -17,14 +17,18 @@ package op import ( - "fmt" + "path/filepath" + "strings" ) // Runs the given Maven command on a specified POM file. Considering the input, this is equivalent to: // mvn {cmd} -f {pomDir}/pom.xml {args...} func RunMavenOnPom(pomDir string, cmd string, args ...string) error { - wa := []string{cmd, "-f", fmt.Sprintf("%s/pom.xml", pomDir)} - return RunCmdAndStreamOutput("mvn", append(wa, args...)) + fullArgs := strings.Split(cmd, " ") + fullArgs = append(fullArgs, "-f", filepath.Join(pomDir, "pom.xml")) + fullArgs = append(fullArgs, args...) + + return RunCmdAndStreamOutput("mvn", fullArgs) } // Runs the given Maven command on a specified module. Considering the input, this is equivalent to: diff --git a/cicd/internal/op/run-cmd.go b/cicd/internal/op/run-cmd.go index 3b7428937a..a8f37136c3 100644 --- a/cicd/internal/op/run-cmd.go +++ b/cicd/internal/op/run-cmd.go @@ -19,9 +19,15 @@ package op import ( "bufio" "fmt" + "io" "log" "os/exec" "strings" + "sync" +) + +var( + printMu sync.Mutex ) // Runs a command and streams the output rather than waiting for it to complete. @@ -30,12 +36,25 @@ func RunCmdAndStreamOutput(cmd string, args []string) error { op := exec.Command(cmd, args...) stdout, _ := op.StdoutPipe() + stderr, _ := op.StderrPipe() op.Start() - scanner := bufio.NewScanner(stdout) + var wg sync.WaitGroup + wg.Add(2) + go processPipe(stdout, &wg) + go processPipe(stderr, &wg) + wg.Wait() + + return op.Wait() +} + +func processPipe(pipe io.ReadCloser, wg *sync.WaitGroup) { + scanner := bufio.NewScanner(pipe) for scanner.Scan() { + printMu.Lock() fmt.Println(scanner.Text()) + printMu.Unlock() } - return op.Wait() + wg.Done() } diff --git a/cicd/internal/repo/files.go b/cicd/internal/repo/files.go index bd93a22e73..ff75a34fd9 100644 --- a/cicd/internal/repo/files.go +++ b/cicd/internal/repo/files.go @@ -30,15 +30,18 @@ const ( RootDirName = "DataflowTemplates" ) -// Gets all the POM files under `dir`. `dir` is a relative path from the root of the repository. -// So if the root is located at `$HOME/go/src/github.com/GoogleCloudPlatform/DataflowTemplates`, then -// passing `v2` represents `$HOME/go/src/github.com/GoogleCloudPlatform/DataflowTemplates/v2`. +// Gets all the POM files under `dir`, where `dir` is a relative path from the root of the repository. func GetAllPomFiles(dir string) ([]string, error) { - root, e := getRootDir() - if e != nil { - return nil, e + return getAllFilesEqualTo(dir, "pom.xml") +} + +// Handles walking through the filesystem from `dir` to get all files equal to `filename`. +func getAllFilesEqualTo(dir string, filename string) ([]string, error) { + root, err := getRootDir() + if err != nil { + return nil, err } - poms := make([]string, 0) + files := make([]string, 0) var start string if strings.HasPrefix(dir, RootDirName) { @@ -47,22 +50,22 @@ func GetAllPomFiles(dir string) ([]string, error) { start = filepath.Join(root, dir) } - e = filepath.Walk(start, func(path string, info fs.FileInfo, err error) error { - if err != nil { - return err + err = filepath.Walk(start, func(path string, info fs.FileInfo, e error) error { + if e != nil { + return e } - if info.IsDir() || info.Name() != "pom.xml" { + if info.IsDir() || info.Name() != filename { return nil } - poms = append(poms, path) + files = append(files, path) return nil }) - if e != nil { - return nil, e + if err != nil { + return nil, err } - return poms, nil + return files, nil } func getRootDir() (string, error) { diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index 3576963cfb..8b810f74b2 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -30,64 +30,6 @@ const ( ItRoot = "it" ) -// Returns all of the known roots modules. -func GetAllRoots() []string { - return []string{ - ClassicRoot, - ItRoot, - FlexRoot, - } -} - -// Returns a map of roots to their modules. Properties are: -// Key: The root module, equivalent to one of the const values (e.g. ClassicRoot) -// Value: All the submodules, sometimes nested under another parent that is also in the slice -// This could be used in the equivalent command: -// mvn x:y -f {key}/pom.xml -pl {value} -// An empty value indicates no submodules. -func GetModuleMapping() map[string][]string { - m := make(map[string][]string) - m[ClassicRoot] = make([]string, 0) - m[ItRoot] = make([]string, 0) - - flexPoms, err := GetAllPomFiles(FlexRoot) - if err != nil { - // Panicking here seems reasonable, since something is deeply wrong with the filesystem - // if this fails. - panic(err) - } - flexModules := make([]string, len(flexPoms)) - for i := range flexPoms { - if module, err := getModuleFromPomPath(flexPoms[i], FlexRoot); err != nil { - panic(err) - } else { - flexModules[i] = module - } - } - m[FlexRoot] = flexModules - - return m -} - -// Extracts module name from POM path, with `rootModule` being used as the reference for -// the uppermost ancestor. The returned value should be usable with the `-pl` flag in relation -// to the POM file at `rootModule`. -func getModuleFromPomPath(pomPath string, rootModule string) (string, error) { - dir := filepath.Dir(pomPath) - allDirs := strings.Split(dir, string(os.PathSeparator)) - - i := len(allDirs) - for ; i > 0 && allDirs[i-1] != rootModule; i -= 1 { - // Empty intentionally - } - - if i == 0 { - return "", fmt.Errorf("%s is not under %s", pomPath, rootModule) - } - - return strings.Join(allDirs[i:], "/"), nil -} - // Gets all the unique modules for files whose path from the root directory is in `paths`. Example paths: // pom.xml -> Mapped to Classic root // v2/cdc-parent/pom.xml -> Mapped to cdc-parent under Flex Templates @@ -126,36 +68,6 @@ func GetModulesForPaths(paths []string) map[string][]string { return m } -type moduleTrieNode struct { - value string - children map[rune]*moduleTrieNode -} - -func flexModulesAsTrie() *moduleTrieNode { - root := &moduleTrieNode{ - value: "", - children: make(map[rune]*moduleTrieNode), - } - - for _, m := range GetModuleMapping()[FlexRoot] { - curr := root - for _, r := range m { - if _, ok := curr.children[r]; ok { - curr = curr.children[r] - } else { - curr.children[r] = &moduleTrieNode{ - value: "", - children: make(map[rune]*moduleTrieNode), - } - curr = curr.children[r] - } - } - curr.value = m - } - - return root -} - func findUniqueFlexModules(paths []string) []string { trie := flexModulesAsTrie() allModules := make([]string, 0) @@ -195,3 +107,78 @@ func findUniqueFlexModules(paths []string) []string { return ret } + +type moduleTrieNode struct { + value string + children map[rune]*moduleTrieNode +} + +func flexModulesAsTrie() *moduleTrieNode { + root := &moduleTrieNode{ + value: "", + children: make(map[rune]*moduleTrieNode), + } + + for _, m := range getModuleMapping()[FlexRoot] { + curr := root + for _, r := range m { + if _, ok := curr.children[r]; ok { + curr = curr.children[r] + } else { + curr.children[r] = &moduleTrieNode{ + value: "", + children: make(map[rune]*moduleTrieNode), + } + curr = curr.children[r] + } + } + curr.value = m + } + + return root +} + +// Returns a map of roots to their modules. Properties are: +// Key: The root module, equivalent to one of the const values (e.g. ClassicRoot) +// Value: All the submodules, sometimes nested under another parent that is also in the slice +func getModuleMapping() map[string][]string { + m := make(map[string][]string) + m[ClassicRoot] = make([]string, 0) + + flexPoms, err := GetAllPomFiles(FlexRoot) + if err != nil { + // Panicking here seems reasonable, since something is deeply wrong with the filesystem + // if this fails. + panic(err) + } + flexModules := make([]string, len(flexPoms)) + for i := range flexPoms { + if module, err := getModuleFromPomPath(flexPoms[i], FlexRoot); err != nil { + panic(err) + } else { + flexModules[i] = module + } + } + m[FlexRoot] = flexModules + + return m +} + +// Extracts module name from POM path, with `rootModule` being used as the reference for +// the uppermost ancestor. The returned value should be usable with the `-pl` flag in relation +// to the POM file at `rootModule`. +func getModuleFromPomPath(pomPath string, rootModule string) (string, error) { + dir := filepath.Dir(pomPath) + allDirs := strings.Split(dir, string(os.PathSeparator)) + + i := len(allDirs) + for ; i > 0 && allDirs[i-1] != rootModule; i -= 1 { + // Empty intentionally + } + + if i == 0 { + return "", fmt.Errorf("%s is not under %s", pomPath, rootModule) + } + + return strings.Join(allDirs[i:], "/"), nil +} diff --git a/cicd/internal/repo/modules_test.go b/cicd/internal/repo/modules_test.go index 903b85e816..ff3876a5c8 100644 --- a/cicd/internal/repo/modules_test.go +++ b/cicd/internal/repo/modules_test.go @@ -21,16 +21,8 @@ import ( "testing" ) -func TestGetAllRoots(t *testing.T) { - actual := GetAllRoots() - expected := []string{ClassicRoot, ItRoot, FlexRoot} - if !reflect.DeepEqual(actual, expected) { - t.Errorf("Not all roots present. Expected: %v. Got: %v", expected, actual) - } -} - func TestModuleMappingHasAllRoots(t *testing.T) { - m := GetModuleMapping() + m := getModuleMapping() if _, ok := m[ClassicRoot]; !ok { t.Error("Missing Classic root") } From f3d5ca374c9f351a338ed92319ebf75fe0685a77 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Sun, 13 Feb 2022 15:26:01 -0500 Subject: [PATCH 089/145] Filter to select files to better decide when to run --- .../{preconditions.yml => java-pr.yml} | 26 +++++++++-- cicd/cmd/run-build/main.go | 25 ++++++----- cicd/cmd/run-spotless/main.go | 3 +- cicd/internal/flags/common-flags.go | 33 +++++++++++++- cicd/internal/flags/common-flags_test.go | 44 ++++++++++++++++++- cicd/internal/op/maven.go | 2 +- cicd/internal/op/run-cmd.go | 2 +- 7 files changed, 115 insertions(+), 20 deletions(-) rename .github/workflows/{preconditions.yml => java-pr.yml} (58%) diff --git a/.github/workflows/preconditions.yml b/.github/workflows/java-pr.yml similarity index 58% rename from .github/workflows/preconditions.yml rename to .github/workflows/java-pr.yml index c3729682aa..36cc0afb55 100644 --- a/.github/workflows/preconditions.yml +++ b/.github/workflows/java-pr.yml @@ -12,13 +12,21 @@ # See the License for the specific language governing permissions and # limitations under the License. -# Checks that don't run any template code and only on PRs. +# Checks that are intended to run on PRs containing Java code. -name: Preconditions +name: Java PR on: pull_request: - branches: ['main'] + branches: + - 'main' + paths: + - '**.java' + - '**pom.xml' + # Include relevant GitHub Action files for running these checks. + # This will make it easier to verify action changes don't break anything. + - '.github/actions/setup-env/*' + - '.github/workflows/java-pr.yml' permissions: read-all @@ -37,3 +45,15 @@ jobs: java-version: '11' # Spotless won't work on version 8 - name: Run Spotless run: ./cicd/run-spotless --changed-files="${{ steps.setup-env.outputs.changed-files }}" + java_build: + name: Build + timeout-minutes: 60 + runs-on: ubuntu-latest + steps: + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Environment + id: setup-env + uses: ./.github/actions/setup-env + - name: Run Build + run: ./cicd/run-build --changed-files="${{ steps.setup-env.outputs.changed-files }}" diff --git a/cicd/cmd/run-build/main.go b/cicd/cmd/run-build/main.go index b70dc65213..dbdb87ca97 100644 --- a/cicd/cmd/run-build/main.go +++ b/cicd/cmd/run-build/main.go @@ -20,7 +20,7 @@ import ( "flag" "log" "strings" - + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/flags" "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" @@ -28,16 +28,21 @@ import ( const ( BuildCommand = "clean install" - FlexPom = "v2/pom.xml" + FlexPom = "v2/pom.xml" ) var ( args = []string{ - "-am", // Include anything this depends on for more reliable builds - "-amd", // Include anything that depends on this to guarantee no breaking changes - "-Dmaven.test.skip", // Skip tests for verifying this builds - "-Dmdep.analyze.skip", // Skip analyzing for dependency failings - "-Djib.skip", // Skip Jib, because don't care about images + "-am", // Include anything this depends on for more reliable builds + "-amd", // Include anything that depends on this to guarantee no breaking changes + "-Dmaven.test.skip", // Skip tests for verifying this builds + "-Dmdep.analyze.skip", // TODO(zhoufek): Fix our dependencies then remove this flag + "-Djib.skip", // Skip Jib, because don't care about images + } + + regexes = []string{ + "\\.java$", + "pom\\.xml$", } ) @@ -45,7 +50,7 @@ func main() { flags.RegisterCommonFlags() flag.Parse() - changed := flags.ChangedFiles() + changed := flags.ChangedFiles(regexes...) if len(changed) == 0 { return } @@ -101,6 +106,6 @@ func removeRoot(flexModules []string) []string { // Order doesn't matter when passing the modules l := len(flexModules) - flexModules[i] = flexModules[l - 1] - return flexModules[:l - 1] + flexModules[i] = flexModules[l-1] + return flexModules[:l-1] } diff --git a/cicd/cmd/run-spotless/main.go b/cicd/cmd/run-spotless/main.go index 9d36e071c0..17dc9398db 100644 --- a/cicd/cmd/run-spotless/main.go +++ b/cicd/cmd/run-spotless/main.go @@ -27,6 +27,7 @@ import ( ) const ( + FileRegex = "\\.(md|java)$" SpotlessCommand = "spotless:check" ) @@ -34,7 +35,7 @@ func main() { flags.RegisterCommonFlags() flag.Parse() - changed := flags.ChangedFiles() + changed := flags.ChangedFiles(FileRegex) if len(changed) == 0 { return } diff --git a/cicd/internal/flags/common-flags.go b/cicd/internal/flags/common-flags.go index 7a6e4053e6..faf965f00f 100644 --- a/cicd/internal/flags/common-flags.go +++ b/cicd/internal/flags/common-flags.go @@ -18,7 +18,9 @@ package flags import ( "flag" + "fmt" "log" + "regexp" "strings" ) @@ -27,15 +29,42 @@ var ( changedFiles string ) +// Registers all common flags. Must be called before flag.Parse(). func RegisterCommonFlags() { flag.StringVar(&changedFiles, "changed-files", "", "List of changed files as a comma-separated string") } -func ChangedFiles() []string { +// Returns all changed files with regexes. If no regexes are passed, all files are returned. If even one +// is passed, then only file paths with a match anywhere in the file will be returned. If multiple are +// passed, it is equivalent to (regex1|regex2|...|regexN) +func ChangedFiles(regexes ...string) []string { if len(changedFiles) == 0 { log.Println("WARNING: No changed files were passed. This could indicate an error.") return make([]string, 0) } - return strings.Split(changedFiles, ",") + files := strings.Split(changedFiles, ",") + if len(regexes) == 0 { + return files + } + + var fullRegex string + if len(regexes) == 1 { + fullRegex = regexes[0] + } else { + fullRegex = fmt.Sprintf("(%s)", strings.Join(regexes, "|")) + } + re := regexp.MustCompile(fullRegex) + + results := make([]string, 0) + for _, f := range files { + if re.MatchString(f) { + results = append(results, f) + } + } + + if len(results) == 0 { + log.Println("INFO: All changed files got filtered out.") + } + return results } diff --git a/cicd/internal/flags/common-flags_test.go b/cicd/internal/flags/common-flags_test.go index 200cd938c5..4df4d4e4d6 100644 --- a/cicd/internal/flags/common-flags_test.go +++ b/cicd/internal/flags/common-flags_test.go @@ -21,7 +21,7 @@ import ( "testing" ) -func TestChangedFiles(t *testing.T) { +func TestChangedFilesNoRegex(t *testing.T) { tests := []struct { input string expected []string @@ -45,10 +45,50 @@ func TestChangedFiles(t *testing.T) { } } -func TestChangedFilesEmpty(t *testing.T) { +func TestChangedFilesNoRegexEmpty(t *testing.T) { changedFiles = "" actual := ChangedFiles() if len(actual) != 0 { t.Errorf("Expected empty slice, but got %v of len %v", actual, len(actual)) } } + +func TestChangedFilesRegexes(t *testing.T) { + tests := []struct { + files string + regexes []string + expected []string + }{ + { + files: "file1,file2,file3", + regexes: []string{"file[1|3]"}, + expected: []string{"file1", "file3"}, + }, + { + files: "file1,file2,file3", + regexes: []string{"f.+1", "fi.+3"}, + expected: []string{"file1", "file3"}, + }, + { + files: "file1,file2,fileN", + regexes: []string{"\\d"}, + expected: []string{"file1", "file2"}, + }, + } + + for _, test := range tests { + changedFiles = test.files + actual := ChangedFiles(test.regexes...) + if !reflect.DeepEqual(actual, test.expected) { + t.Errorf("Returned files are not equal. Expected %v. Got %v.", test.expected, actual) + } + } +} + +func TestChangedFilesRegexesNoMatch(t *testing.T) { + changedFiles = "foo,bar" + actual := ChangedFiles("file") + if len(actual) != 0 { + t.Errorf("Expected empty slice but got %v", actual) + } +} diff --git a/cicd/internal/op/maven.go b/cicd/internal/op/maven.go index 09e1f1d8af..064e881d7d 100644 --- a/cicd/internal/op/maven.go +++ b/cicd/internal/op/maven.go @@ -27,7 +27,7 @@ func RunMavenOnPom(pomDir string, cmd string, args ...string) error { fullArgs := strings.Split(cmd, " ") fullArgs = append(fullArgs, "-f", filepath.Join(pomDir, "pom.xml")) fullArgs = append(fullArgs, args...) - + return RunCmdAndStreamOutput("mvn", fullArgs) } diff --git a/cicd/internal/op/run-cmd.go b/cicd/internal/op/run-cmd.go index a8f37136c3..e877638b73 100644 --- a/cicd/internal/op/run-cmd.go +++ b/cicd/internal/op/run-cmd.go @@ -26,7 +26,7 @@ import ( "sync" ) -var( +var ( printMu sync.Mutex ) From 69325e3b79fafe3c1454906b3d694b7edd00fc5e Mon Sep 17 00:00:00 2001 From: dhercher Date: Mon, 28 Mar 2022 14:40:55 -0700 Subject: [PATCH 090/145] DatastreamToBigQuery: adding datastream logging for json failures PiperOrigin-RevId: 437858375 --- .../teleport/v2/transforms/FormatDatastreamJsonToJson.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java index ed76410d74..f018f82c68 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/FormatDatastreamJsonToJson.java @@ -64,8 +64,8 @@ record = new ObjectMapper().readTree(c.element()); } } } catch (IOException e) { - LOG.error("Issue parsing JSON record. Unable to continue.", e); - throw new RuntimeException(e); + LOG.error("Skipping Malformed JSON record: {} -> {}", c.element(), e.getMessage()); + return; } ObjectMapper mapper = new ObjectMapper(); From 73717758b54640311823d03378d9bec8e6cf68a9 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Wed, 23 Feb 2022 15:44:05 -0500 Subject: [PATCH 091/145] Add common workflow and unit tests --- .github/actions/setup-env/action.yml | 6 + .github/workflows/java-pr.yml | 16 ++ .github/workflows/prepare-java-cache.yml | 14 +- cicd/cmd/run-build/main.go | 96 ++--------- cicd/cmd/run-spotless/main.go | 41 +---- cicd/cmd/run-unit-tests/main.go | 37 +++++ cicd/internal/flags/common-flags_test.go | 5 + cicd/internal/op/maven.go | 16 +- cicd/internal/repo/modules.go | 1 + cicd/internal/workflows/maven-workflows.go | 185 +++++++++++++++++++++ cicd/internal/workflows/workflow.go | 22 +++ 11 files changed, 302 insertions(+), 137 deletions(-) create mode 100644 cicd/cmd/run-unit-tests/main.go create mode 100644 cicd/internal/workflows/maven-workflows.go create mode 100644 cicd/internal/workflows/workflow.go diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index f28f3a6fdb..6fdd8cfbe5 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -26,6 +26,11 @@ name: 'Setup Environment' description: 'Sets up common environment for Dataflow Templates workflows' inputs: + java-cache-key: + type: string + description: 'Key to use for the cache entry' + required: false + default: '' java-version: type: string description: 'The version of Java to install' @@ -66,4 +71,5 @@ runs: - name: Setup Java uses: ./.github/actions/setup-java-env with: + cache-key: ${{ inputs.java-cache-key }} java-version: ${{ inputs.java-version }} diff --git a/.github/workflows/java-pr.yml b/.github/workflows/java-pr.yml index 36cc0afb55..084a0911c6 100644 --- a/.github/workflows/java-pr.yml +++ b/.github/workflows/java-pr.yml @@ -57,3 +57,19 @@ jobs: uses: ./.github/actions/setup-env - name: Run Build run: ./cicd/run-build --changed-files="${{ steps.setup-env.outputs.changed-files }}" + - name: Cleanup Java Environment + uses: ./.github/actions/cleanup-java-env + java_tests: + name: Unit Tests + timeout-minutes: 60 + runs-on: ubuntu-latest + steps: + - name: Checkout Code + uses: actions/checkout@230611dbd0eb52da1e1f4f7bc8bb0c3a339fc8b7 + - name: Setup Environment + id: setup-env + uses: ./.github/actions/setup-env + - name: Run Unit Tests + run: ./cicd/run-unit-tests --changed-files="${{ steps.setup-env.outputs.changed-files }}" + - name: Cleanup Java Environment + uses: ./.github/actions/cleanup-java-env diff --git a/.github/workflows/prepare-java-cache.yml b/.github/workflows/prepare-java-cache.yml index f111ed40dd..ef3ad3b3bd 100644 --- a/.github/workflows/prepare-java-cache.yml +++ b/.github/workflows/prepare-java-cache.yml @@ -37,7 +37,7 @@ on: branches: - 'main' paths: - - '.github/workflows/prepare-java-chace.yml' + - '.github/workflows/prepare-java-cache.yml' - '.github/actions/setup-java-env/*' - '.github/actions/cleanup-java-env/*' @@ -67,11 +67,11 @@ jobs: - name: Resolve Dependencies if: ${{ steps.setup-java.outputs.cache-hit != 'true' || github.event_name == 'pull_request' }} run: | - for DIR in $(find . -maxdepth 1 -type d); do - POMF="$DIR/pom.xml" - if [[ -f "$POMF" ]]; then - mvn -B clean install -f "$POMF" -am -amd -Dmaven.test.skip -Dcheckstyle.skip -Djib.skip -Dmdep.analyze.skip - fi - done + mvn -B clean install -f unified-templates.xml \ + -Dmaven.test.skip \ + -Dcheckstyle.skip \ + -Djib.skip \ + -Dmdep.analyze.skip \ + -Dspotless.check.skip - name: Cleanup Java Environment uses: ./.github/actions/cleanup-java-env diff --git a/cicd/cmd/run-build/main.go b/cicd/cmd/run-build/main.go index dbdb87ca97..aa89bc9e13 100644 --- a/cicd/cmd/run-build/main.go +++ b/cicd/cmd/run-build/main.go @@ -17,95 +17,21 @@ package main import ( - "flag" "log" - "strings" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/flags" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" -) - -const ( - BuildCommand = "clean install" - FlexPom = "v2/pom.xml" -) - -var ( - args = []string{ - "-am", // Include anything this depends on for more reliable builds - "-amd", // Include anything that depends on this to guarantee no breaking changes - "-Dmaven.test.skip", // Skip tests for verifying this builds - "-Dmdep.analyze.skip", // TODO(zhoufek): Fix our dependencies then remove this flag - "-Djib.skip", // Skip Jib, because don't care about images - } - - regexes = []string{ - "\\.java$", - "pom\\.xml$", - } + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/workflows" ) func main() { - flags.RegisterCommonFlags() - flag.Parse() - - changed := flags.ChangedFiles(regexes...) - if len(changed) == 0 { - return - } - - for root, children := range repo.GetModulesForPaths(changed) { - var err error - if root == repo.ClassicRoot { - err = op.RunMavenOnPom(root, BuildCommand, args...) - } else if root == repo.FlexRoot { - // A change to the root POM could impact all Flex Templates, so we must rebuild everything on - // those changes. - if buildAllFlexTemplates(changed) { - log.Println("A change indicated to build all Flex Modules. This may take multiple minutes.") - err = op.RunMavenOnPom(root, BuildCommand, args...) - } else { - children = removeRoot(children) - if len(children) != 0 { - err = op.RunMavenOnModule(root, BuildCommand, strings.Join(children, ","), args...) - } else { - log.Println("The only Flex changes were to files that should not trigger a build") - } - } - } - - if err != nil { - log.Fatalf("%v", err) - } - } -} - -// Returns true if all flex templates need to be built. Otherwise, false. -func buildAllFlexTemplates(changed []string) bool { - for _, c := range changed { - if c == FlexPom { - return true - } - } - return false -} - -// Removes root and returns results. This may reorder the input. -func removeRoot(flexModules []string) []string { - var i int - for i = 0; i < len(flexModules); i += 1 { - if flexModules[i] == "" { - break - } + mvnFlags := workflows.NewMavenFlags() + err := workflows.MvnCleanInstall().Run( + mvnFlags.IncludeDependencies(), + mvnFlags.IncludeDependents(), + mvnFlags.SkipDependencyAnalysis(), // TODO(zhoufek): Fix our dependencies then remove this flag + mvnFlags.SkipJib(), + mvnFlags.SkipTests()) + if err != nil { + log.Fatalf("%v\n", err) } - - if i == len(flexModules) { - return flexModules - } - - // Order doesn't matter when passing the modules - l := len(flexModules) - flexModules[i] = flexModules[l-1] - return flexModules[:l-1] + log.Println("Build Successful!") } diff --git a/cicd/cmd/run-spotless/main.go b/cicd/cmd/run-spotless/main.go index 17dc9398db..40b57a873e 100644 --- a/cicd/cmd/run-spotless/main.go +++ b/cicd/cmd/run-spotless/main.go @@ -17,46 +17,13 @@ package main import ( - "flag" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/workflows" "log" - "strings" - - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/flags" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" - "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" -) - -const ( - FileRegex = "\\.(md|java)$" - SpotlessCommand = "spotless:check" ) func main() { - flags.RegisterCommonFlags() - flag.Parse() - - changed := flags.ChangedFiles(FileRegex) - if len(changed) == 0 { - return - } - - errored := false - for root, children := range repo.GetModulesForPaths(changed) { - var err error - if len(children) == 0 { - err = op.RunMavenOnPom(root, SpotlessCommand) - } else if len(children) > 1 || children[0] != "" { - err = op.RunMavenOnModule(root, SpotlessCommand, strings.Join(children, ",")) - } else { - log.Printf("Skipping '%s' because the only files changed were not associated with a module", root) - } - - if err != nil { - errored = true - } - } - - if errored { - log.Fatal("There were spotless errors. Check the output from the commands.") + if err := workflows.SpotlessCheck().Run(); err != nil { + log.Fatalf("Error running spotless check: %v", err) } + log.Println("Spotless check completed successfully!") } diff --git a/cicd/cmd/run-unit-tests/main.go b/cicd/cmd/run-unit-tests/main.go new file mode 100644 index 0000000000..f7d17e0e57 --- /dev/null +++ b/cicd/cmd/run-unit-tests/main.go @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2022 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 main + +import ( + "log" + + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/workflows" +) + +func main() { + mvnFlags := workflows.NewMavenFlags() + err := workflows.MvnCleanInstall().Run( + mvnFlags.IncludeDependencies(), + mvnFlags.IncludeDependents(), + mvnFlags.SkipCheckstyle(), + mvnFlags.SkipDependencyAnalysis(), + mvnFlags.SkipJib()) + if err != nil { + log.Fatalf("%v\n", err) + } + log.Println("Build Successful!") +} diff --git a/cicd/internal/flags/common-flags_test.go b/cicd/internal/flags/common-flags_test.go index 4df4d4e4d6..0881b1bb21 100644 --- a/cicd/internal/flags/common-flags_test.go +++ b/cicd/internal/flags/common-flags_test.go @@ -74,6 +74,11 @@ func TestChangedFilesRegexes(t *testing.T) { regexes: []string{"\\d"}, expected: []string{"file1", "file2"}, }, + { + files: "foo.c,bar.cc", + regexes: []string{"\\.c$"}, + expected: []string{"foo.c"}, + }, } for _, test := range tests { diff --git a/cicd/internal/op/maven.go b/cicd/internal/op/maven.go index 064e881d7d..a8e35ebfca 100644 --- a/cicd/internal/op/maven.go +++ b/cicd/internal/op/maven.go @@ -17,24 +17,24 @@ package op import ( - "path/filepath" "strings" ) // Runs the given Maven command on a specified POM file. Considering the input, this is equivalent to: -// mvn {cmd} -f {pomDir}/pom.xml {args...} -func RunMavenOnPom(pomDir string, cmd string, args ...string) error { - fullArgs := strings.Split(cmd, " ") - fullArgs = append(fullArgs, "-f", filepath.Join(pomDir, "pom.xml")) +// mvn -B {cmd} -f {pom} {args...} +func RunMavenOnPom(pom string, cmd string, args ...string) error { + fullArgs := []string{"-B"} + fullArgs = append(fullArgs, strings.Split(cmd, " ")...) + fullArgs = append(fullArgs, "-f", pom) fullArgs = append(fullArgs, args...) return RunCmdAndStreamOutput("mvn", fullArgs) } // Runs the given Maven command on a specified module. Considering the input, this is equivalent to: -// mvn {cmd} -f {pomDir}/pom.xml -pl {module} {args...} -func RunMavenOnModule(pomDir string, cmd string, module string, args ...string) error { +// mvn -B {cmd} -f {pom} -pl {module} {args...} +func RunMavenOnModule(pom string, cmd string, module string, args ...string) error { fullArgs := []string{"-pl", module} fullArgs = append(fullArgs, args...) - return RunMavenOnPom(pomDir, cmd, fullArgs...) + return RunMavenOnPom(pom, cmd, fullArgs...) } diff --git a/cicd/internal/repo/modules.go b/cicd/internal/repo/modules.go index 8b810f74b2..4a4950e9fd 100644 --- a/cicd/internal/repo/modules.go +++ b/cicd/internal/repo/modules.go @@ -144,6 +144,7 @@ func flexModulesAsTrie() *moduleTrieNode { func getModuleMapping() map[string][]string { m := make(map[string][]string) m[ClassicRoot] = make([]string, 0) + m[ItRoot] = make([]string, 0) flexPoms, err := GetAllPomFiles(FlexRoot) if err != nil { diff --git a/cicd/internal/workflows/maven-workflows.go b/cicd/internal/workflows/maven-workflows.go new file mode 100644 index 0000000000..378e8f9156 --- /dev/null +++ b/cicd/internal/workflows/maven-workflows.go @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2022 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 workflows + +import ( + "flag" + "fmt" + "log" + "path/filepath" + "strings" + + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/flags" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/op" + "github.com/GoogleCloudPlatform/DataflowTemplates/cicd/internal/repo" +) + +const ( + // mvn commands + cleanInstallCmd = "clean install" + spotlessCheckCmd = "spotless:check" + + // regexes + javaFileRegex = "\\.java$" + markdownFileRegex = "\\.md$" + pomFileRegex = "pom\\.xml$" + + // notable files + unifiedPom = "unified-templates.xml" +) + +// Interface for retrieving flags that can be passed into the workflow's +// `Run` method. +type MavenFlags interface { + IncludeDependencies() string + IncludeDependents() string + SkipCheckstyle() string + SkipDependencyAnalysis() string + SkipJib() string + SkipTests() string +} + +type mvnFlags struct{} + +func (*mvnFlags) IncludeDependencies() string { + return "-am" +} + +func (*mvnFlags) IncludeDependents() string { + return "-amd" +} + +func (*mvnFlags) SkipCheckstyle() string { + return "-Dcheckstyle.skip" +} + +func (*mvnFlags) SkipDependencyAnalysis() string { + return "-Dmdep.analyze.skip" +} + +func (*mvnFlags) SkipJib() string { + return "-Djib.skip" +} + +func (*mvnFlags) SkipTests() string { + return "-Dmaven.test.skip" +} + +func NewMavenFlags() MavenFlags { + return &mvnFlags{} +} + +type mvnCleanInstallWorkflow struct{} + +func MvnCleanInstall() Workflow { + return &mvnCleanInstallWorkflow{} +} + +func (*mvnCleanInstallWorkflow) Run(args ...string) error { + flags.RegisterCommonFlags() + flag.Parse() + + changed := flags.ChangedFiles(javaFileRegex, pomFileRegex) + if len(changed) == 0 { + return nil + } + + // Collect the modules together for a single call. Maven can work out the install order. + modules := make([]string, 0) + for root, children := range repo.GetModulesForPaths(changed) { + if len(children) == 0 { + modules = append(modules, root) + continue + } + + // A change to the root POM could impact all children, so build them all. + buildAll := false + for _, c := range changed { + if c == filepath.Join(root, "pom.xml") { + buildAll = true + break + } + } + if buildAll { + modules = append(modules, root) + continue + } + + withoutRoot := removeRoot(children) + if len(withoutRoot) == 0 { + log.Printf("All files under %s were irrelevant root-level files", root) + } + for _, m := range withoutRoot { + modules = append(modules, fmt.Sprintf("%s/%s", root, m)) + } + } + + if len(modules) == 0 { + log.Println("All modules were filtered out.") + return nil + } + + return op.RunMavenOnModule(unifiedPom, cleanInstallCmd, strings.Join(modules, ","), args...) +} + +type spotlessCheckWorkflow struct{} + +func SpotlessCheck() Workflow { + return &spotlessCheckWorkflow{} +} + +func (*spotlessCheckWorkflow) Run(args ...string) error { + flags.RegisterCommonFlags() + flag.Parse() + + changed := flags.ChangedFiles(javaFileRegex, markdownFileRegex) + if len(changed) == 0 { + return nil + } + + modules := make([]string, 0) + for root, children := range repo.GetModulesForPaths(changed) { + if len(children) == 0 || (len(children) == 1 && children[0] == "") { + modules = append(modules, root) + continue + } + for _, c := range children { + modules = append(modules, fmt.Sprintf("%s/%s", root, c)) + } + } + + return op.RunMavenOnModule(unifiedPom, spotlessCheckCmd, strings.Join(modules, ","), args...) +} + +// Removes root and returns results. This may reorder the input. +func removeRoot(modules []string) []string { + var i int + for i = 0; i < len(modules); i += 1 { + if modules[i] == "" { + break + } + } + + if i == len(modules) { + return modules + } + + // Order doesn't matter when passing the modules + l := len(modules) + modules[i] = modules[l-1] + return modules[:l-1] +} diff --git a/cicd/internal/workflows/workflow.go b/cicd/internal/workflows/workflow.go new file mode 100644 index 0000000000..00dcb91c45 --- /dev/null +++ b/cicd/internal/workflows/workflow.go @@ -0,0 +1,22 @@ +/* + * Copyright (C) 2022 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 workflows + +type Workflow interface { + // Runs a workflow. This should handle registering and parsing all flags. + Run(args ...string) error +} From b0e3131f7f89da84ba002a9c676def5a7ba5f0e3 Mon Sep 17 00:00:00 2001 From: dhercher Date: Thu, 31 Mar 2022 03:19:19 -0700 Subject: [PATCH 092/145] DatastreamToSQL: improved logging PiperOrigin-RevId: 438522759 --- .../cloud/teleport/v2/utils/DatastreamToDML.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java index b6b56c1750..2da44af077 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java @@ -211,8 +211,7 @@ public DmlInfo convertJsonToDmlInfo(JsonNode rowObj, String failsafeValue) { Map tableSchema = this.getTableSchema(catalogName, schemaName, tableName); if (tableSchema.isEmpty()) { - // If the table DNE we return null (NOOP) - LOG.debug("Table Not Found: {}.{}.{}", catalogName, schemaName, tableName); + // If the table DNE we return null (NOOP). return null; } @@ -496,6 +495,13 @@ private Map getTableSchema( e.toString()); } + if (tableSchema.isEmpty()) { + LOG.info( + "Table Not Found: Catalog: {}, Schema: {}, Table: {}", + catalogName, + schemaName, + tableName); + } return tableSchema; } From cb0b364a06c4b03bd1032f20777803f51272d3cd Mon Sep 17 00:00:00 2001 From: weiwenxu Date: Fri, 1 Apr 2022 13:42:57 -0700 Subject: [PATCH 093/145] Fix bug to get specific Java object from JDBC data type Date and timestamp to avoid errors from the formatter. PiperOrigin-RevId: 438901228 --- .../cloud/teleport/templates/common/JdbcConverters.java | 4 ++-- .../cloud/teleport/templates/common/JdbcConvertersTest.java | 2 ++ .../com/google/cloud/teleport/v2/utils/JdbcConverters.java | 4 ++-- .../google/cloud/teleport/v2/utils/JdbcConvertersTest.java | 2 ++ 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java index 938b5b1f32..24cc3f8326 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java @@ -140,7 +140,7 @@ public TableRow mapRow(ResultSet resultSet) throws Exception { switch (metaData.getColumnTypeName(i).toLowerCase()) { case "date": outputTableRow.set( - metaData.getColumnName(i), dateFormatter.format(resultSet.getObject(i))); + metaData.getColumnName(i), dateFormatter.format(resultSet.getDate(i))); break; case "datetime": outputTableRow.set( @@ -149,7 +149,7 @@ public TableRow mapRow(ResultSet resultSet) throws Exception { break; case "timestamp": outputTableRow.set( - metaData.getColumnName(i), timestampFormatter.format(resultSet.getObject(i))); + metaData.getColumnName(i), timestampFormatter.format(resultSet.getTimestamp(i))); break; default: outputTableRow.set(metaData.getColumnName(i), resultSet.getObject(i)); diff --git a/src/test/java/com/google/cloud/teleport/templates/common/JdbcConvertersTest.java b/src/test/java/com/google/cloud/teleport/templates/common/JdbcConvertersTest.java index fd1a0def02..dc7f1cb185 100644 --- a/src/test/java/com/google/cloud/teleport/templates/common/JdbcConvertersTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/common/JdbcConvertersTest.java @@ -89,7 +89,9 @@ public void testTemporalFields() throws Exception { Mockito.when(resultSet.getObject(1)).thenReturn(datetimeObj); Mockito.when(resultSet.getObject(2)).thenReturn(dateObj); + Mockito.when(resultSet.getDate(2)).thenReturn(dateObj); Mockito.when(resultSet.getObject(3)).thenReturn(timestampObj); + Mockito.when(resultSet.getTimestamp(3)).thenReturn(timestampObj); Mockito.when(resultSet.getMetaData()).thenReturn(resultSetMetaData); Mockito.when(resultSetMetaData.getColumnCount()).thenReturn(3); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java index 9909dbfce0..9bf97dcbf4 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java @@ -65,7 +65,7 @@ public TableRow mapRow(ResultSet resultSet) throws Exception { switch (metaData.getColumnTypeName(i).toLowerCase()) { case "date": outputTableRow.set( - metaData.getColumnName(i), dateFormatter.format(resultSet.getObject(i))); + metaData.getColumnName(i), dateFormatter.format(resultSet.getDate(i))); break; case "datetime": outputTableRow.set( @@ -74,7 +74,7 @@ public TableRow mapRow(ResultSet resultSet) throws Exception { break; case "timestamp": outputTableRow.set( - metaData.getColumnName(i), timestampFormatter.format(resultSet.getObject(i))); + metaData.getColumnName(i), timestampFormatter.format(resultSet.getTimestamp(i))); break; default: outputTableRow.set(metaData.getColumnName(i), resultSet.getObject(i)); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/JdbcConvertersTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/JdbcConvertersTest.java index 4b947202cb..e1fae1a77c 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/JdbcConvertersTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/JdbcConvertersTest.java @@ -93,7 +93,9 @@ public void testTemporalFields() throws Exception { Mockito.when(resultSet.getObject(1)).thenReturn(datetimeObj); Mockito.when(resultSet.getObject(2)).thenReturn(dateObj); + Mockito.when(resultSet.getDate(2)).thenReturn(dateObj); Mockito.when(resultSet.getObject(3)).thenReturn(timestampObj); + Mockito.when(resultSet.getTimestamp(3)).thenReturn(timestampObj); Mockito.when(resultSet.getMetaData()).thenReturn(resultSetMetaData); Mockito.when(resultSetMetaData.getColumnCount()).thenReturn(3); From 8c96ef4fd875759435ae48ad881bf0a0f2af96a5 Mon Sep 17 00:00:00 2001 From: weiwenxu Date: Fri, 1 Apr 2022 15:15:23 -0700 Subject: [PATCH 094/145] Fix bug that handles JDBC CLOB data converting to String PiperOrigin-RevId: 438922034 --- .../templates/common/JdbcConverters.java | 16 +++++++++++++ .../teleport/v2/templates/JdbcToPubsub.java | 24 +++++++++++++++++-- .../teleport/v2/utils/JdbcConverters.java | 16 +++++++++++++ .../v2/templates/JdbcToPubsubTest.java | 13 +++++++++- 4 files changed, 66 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java index 24cc3f8326..0060669e5f 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.templates.common; import com.google.api.services.bigquery.model.TableRow; +import java.sql.Clob; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.text.SimpleDateFormat; @@ -25,10 +26,14 @@ import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Common code for Teleport JdbcToBigQuery. */ public class JdbcConverters { + private static final Logger LOG = LoggerFactory.getLogger(JdbcConverters.class); + /** Interface used by the JdbcToBigQuery pipeline to accept user input. */ public interface JdbcToBigQueryOptions extends PipelineOptions { @@ -151,6 +156,17 @@ public TableRow mapRow(ResultSet resultSet) throws Exception { outputTableRow.set( metaData.getColumnName(i), timestampFormatter.format(resultSet.getTimestamp(i))); break; + case "clob": + Clob clobObject = resultSet.getClob(i); + if (clobObject.length() > Integer.MAX_VALUE) { + LOG.warn( + "The Clob value size {} in column {} exceeds 2GB and will be truncated.", + clobObject.length(), + metaData.getColumnName(i)); + } + outputTableRow.set( + metaData.getColumnName(i), clobObject.getSubString(1, (int) clobObject.length())); + break; default: outputTableRow.set(metaData.getColumnName(i), resultSet.getObject(i)); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java index 690c02f6b1..8d43763892 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java @@ -18,6 +18,7 @@ import com.google.cloud.teleport.v2.io.DynamicJdbcIO; import com.google.cloud.teleport.v2.options.JdbcToPubsubOptions; import com.google.cloud.teleport.v2.utils.KMSEncryptedNestedValue; +import java.sql.Clob; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import org.apache.beam.sdk.Pipeline; @@ -56,10 +57,29 @@ public String mapRow(ResultSet resultSet) throws Exception { for (int i = 1; i <= metaData.getColumnCount(); i++) { Object value = resultSet.getObject(i); + // JSONObject.put() does not support null values. The exception is JSONObject.NULL - json.put(metaData.getColumnLabel(i), value == null ? JSONObject.NULL : value); + if (value == null) { + json.put(metaData.getColumnLabel(i), JSONObject.NULL); + continue; + } + + switch (metaData.getColumnTypeName(i).toLowerCase()) { + case "clob": + Clob clobObject = resultSet.getClob(i); + if (clobObject.length() > Integer.MAX_VALUE) { + LOG.warn( + "The Clob value size {} in column {} exceeds 2GB and will be truncated.", + clobObject.length(), + metaData.getColumnLabel(i)); + } + json.put( + metaData.getColumnLabel(i), clobObject.getSubString(1, (int) clobObject.length())); + break; + default: + json.put(metaData.getColumnLabel(i), value); + } } - return json.toString(); } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java index 9bf97dcbf4..67936d3845 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java @@ -16,16 +16,21 @@ package com.google.cloud.teleport.v2.utils; import com.google.api.services.bigquery.model.TableRow; +import java.sql.Clob; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.text.SimpleDateFormat; import java.time.format.DateTimeFormatter; import java.time.temporal.TemporalAccessor; import org.apache.beam.sdk.io.jdbc.JdbcIO; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Common code for Teleport DataplexJdbcIngestion. */ public class JdbcConverters { + private static final Logger LOG = LoggerFactory.getLogger(JdbcConverters.class); + /** Factory method for {@link ResultSetToTableRow}. */ public static JdbcIO.RowMapper getResultSetToTableRow() { return new ResultSetToTableRow(); @@ -76,6 +81,17 @@ public TableRow mapRow(ResultSet resultSet) throws Exception { outputTableRow.set( metaData.getColumnName(i), timestampFormatter.format(resultSet.getTimestamp(i))); break; + case "clob": + Clob clobObject = resultSet.getClob(i); + if (clobObject.length() > Integer.MAX_VALUE) { + LOG.warn( + "The Clob value size {} in column {} exceeds 2GB and will be truncated.", + clobObject.length(), + metaData.getColumnName(i)); + } + outputTableRow.set( + metaData.getColumnName(i), clobObject.getSubString(1, (int) clobObject.length())); + break; default: outputTableRow.set(metaData.getColumnName(i), resultSet.getObject(i)); } diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToPubsubTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToPubsubTest.java index 98b5e1ada4..0990c876c0 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToPubsubTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToPubsubTest.java @@ -19,6 +19,7 @@ import static org.mockito.Mockito.when; import com.google.cloud.teleport.v2.templates.JdbcToPubsub.ResultSetToJSONString; +import java.sql.Clob; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import org.json.JSONObject; @@ -37,15 +38,24 @@ public final class JdbcToPubsubTest { @Rule public final MockitoRule mockito = MockitoJUnit.rule(); @Mock private ResultSet rs; @Mock private ResultSetMetaData rsMetaData; + @Mock private Clob clobColumn; @Test public void testResultSetToJSONString() throws Exception { - when(rsMetaData.getColumnCount()).thenReturn(2); + when(rsMetaData.getColumnCount()).thenReturn(3); when(rsMetaData.getColumnLabel(1)).thenReturn("Name"); when(rsMetaData.getColumnLabel(2)).thenReturn("Points"); + when(rsMetaData.getColumnLabel(3)).thenReturn("Clob_Column"); + when(rsMetaData.getColumnTypeName(1)).thenReturn("VARCHAR"); + when(rsMetaData.getColumnTypeName(2)).thenReturn("INTEGER"); + when(rsMetaData.getColumnTypeName(3)).thenReturn("CLOB"); when(rs.getMetaData()).thenReturn(rsMetaData); when(rs.getObject(1)).thenReturn("testName"); when(rs.getObject(2)).thenReturn(123); + when(rs.getObject(3)).thenReturn(clobColumn); + when(rs.getClob(3)).thenReturn(clobColumn); + when(clobColumn.length()).thenReturn((long) 20); + when(clobColumn.getSubString(1, 20)).thenReturn("This is a long text."); ResultSetToJSONString mapper = new ResultSetToJSONString(); String jsonString = mapper.mapRow(rs); @@ -53,5 +63,6 @@ public void testResultSetToJSONString() throws Exception { assertEquals(jsonObject.getInt("Points"), 123); assertEquals(jsonObject.getString("Name"), "testName"); + assertEquals(jsonObject.getString("Clob_Column"), "This is a long text."); } } From ab45473c25e52a9ec9c78aa0fe51e18045114830 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Mon, 4 Apr 2022 11:47:24 -0700 Subject: [PATCH 095/145] Add parameter to DatplexJdbcIngestion template which allows the user to specify strategy if destination file exists. PiperOrigin-RevId: 439373567 --- .../options/DataplexJdbcIngestionOptions.java | 18 +- .../v2/templates/DataplexJdbcIngestion.java | 121 +++++++++++- .../GenericRecordsToGcsPartitioned.java | 131 ++----------- .../v2/utils/DataplexJdbcIngestionFilter.java | 132 +++++++++++++ .../v2/utils/DataplexJdbcIngestionNaming.java | 47 +++++ .../v2/utils/DataplexJdbcPartitionUtils.java | 141 ++++++++++++++ .../teleport/v2/utils/JdbcConverters.java | 2 +- .../utils/JdbcIngestionWriteDisposition.java | 76 ++++++++ .../GenericRecordsToGcsPartitionedTest.java | 2 +- .../DataplexJdbcIngestionFilterTest.java | 179 ++++++++++++++++++ 10 files changed, 718 insertions(+), 131 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilter.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionNaming.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcPartitionUtils.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcIngestionWriteDisposition.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilterTest.java diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java index b3d64449c4..f89850387b 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/DataplexJdbcIngestionOptions.java @@ -15,10 +15,10 @@ */ package com.google.cloud.teleport.v2.options; -import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned.PartitioningSchema; +import com.google.cloud.teleport.v2.utils.DataplexJdbcPartitionUtils.PartitioningSchema; import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.WriteDispositionOptions; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; @@ -122,15 +122,13 @@ public interface DataplexJdbcIngestionOptions extends GcpOptions, PipelineOption void setParitionColumn(String partitionColumn); @Description( - "If the table exists - should it overwrite / append or fail the load. " - + "Default: WRITE_APPEND. " - + "This currently only supports writing to BigQuery." - + "Allowed formats are: " - + "WRITE_APPEND / WRITE_TRUNCATE / WRITE_EMPTY") - @Default.Enum("WRITE_APPEND") - WriteDisposition getWriteDisposition(); + "Strategy to employ if the target file/table exists. For BigQuery, allowed formats are:" + + " WRITE_APPEND / WRITE_TRUNCATE / WRITE_EMPTY. For GCS, allowed" + + " formats are: SKIP / WRITE_TRUNCATE / WRITE_EMPTY. Default: WRITE_EMPTY. ") + @Default.Enum("WRITE_EMPTY") + WriteDispositionOptions getWriteDisposition(); - void setWriteDisposition(WriteDisposition writeDisposition); + void setWriteDisposition(WriteDispositionOptions writeDisposition); @Description("Output file format in GCS. Format: PARQUET, AVRO, or ORC. Default: PARQUET.") @Default.Enum("PARQUET") diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexJdbcIngestion.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexJdbcIngestion.java index bee22df6cc..b6c291cf70 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexJdbcIngestion.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/DataplexJdbcIngestion.java @@ -26,13 +26,19 @@ import com.google.cloud.teleport.v2.options.DataplexJdbcIngestionOptions; import com.google.cloud.teleport.v2.transforms.BeamRowToGenericRecordFn; import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned; +import com.google.cloud.teleport.v2.utils.DataplexJdbcIngestionFilter; +import com.google.cloud.teleport.v2.utils.DataplexJdbcIngestionNaming; import com.google.cloud.teleport.v2.utils.JdbcConverters; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.MapWriteDisposition; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.WriteDispositionException; import com.google.cloud.teleport.v2.utils.KMSEncryptedNestedValue; import com.google.cloud.teleport.v2.utils.Schemas; +import com.google.cloud.teleport.v2.utils.StorageUtils; import com.google.cloud.teleport.v2.values.DataplexAssetResourceSpec; import com.google.cloud.teleport.v2.values.PartitionMetadata; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; +import java.util.List; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; @@ -43,9 +49,14 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.utils.AvroUtils; +import org.apache.beam.sdk.transforms.Distinct; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TupleTag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,6 +74,13 @@ public class DataplexJdbcIngestion { /* Logger for class.*/ private static final Logger LOG = LoggerFactory.getLogger(DataplexJdbcIngestion.class); + /** The tag for filtered records. */ + private static final TupleTag FILTERED_RECORDS_OUT = + new TupleTag() {}; + + /** The tag for existing target file names. */ + private static final TupleTag EXISTING_TARGET_FILES_OUT = new TupleTag() {}; + private static KMSEncryptedNestedValue maybeDecrypt(String unencryptedValue, String kmsKey) { return new KMSEncryptedNestedValue(unencryptedValue, kmsKey); } @@ -133,13 +151,96 @@ private static GoogleCloudDataplexV1Asset resolveAsset( return asset; } + private static boolean shouldSkipUnpartitionedTable( + DataplexJdbcIngestionOptions options, String targetRootPath, List existingFiles) { + String expectedFilePath = + new DataplexJdbcIngestionNaming(options.getFileFormat().getFileSuffix()) + .getSingleFilename(); + if (existingFiles.contains(expectedFilePath)) { + // Target file exists, performing writeDispositionGcs strategy + switch (options.getWriteDisposition()) { + case WRITE_EMPTY: + throw new WriteDispositionException( + String.format( + "Target File %s already exists in the output asset bucket %s. Failing" + + " according to writeDisposition.", + expectedFilePath, targetRootPath)); + case SKIP: + LOG.info( + "Target File {} already exists in the output asset bucket {}. Skipping" + + " according to writeDisposition.", + expectedFilePath, + targetRootPath); + return true; + case WRITE_TRUNCATE: + LOG.info( + "Target File {} already exists in the output asset bucket {}. Overwriting" + + " according to writeDisposition.", + expectedFilePath, + targetRootPath); + return false; + default: + throw new UnsupportedOperationException( + options.getWriteDisposition() + + " writeDisposition not implemented for writing to GCS."); + } + } + return false; + } + + private static PCollection applyPartitionedWriteDispositionFilter( + PCollection genericRecords, + DataplexJdbcIngestionOptions options, + String targetRootPath, + org.apache.avro.Schema avroSchema, + List existingFiles) { + PCollectionTuple filteredRecordsTuple = + genericRecords.apply( + "Filter pre-existing records", + new DataplexJdbcIngestionFilter( + targetRootPath, + Schemas.serialize(avroSchema), + options.getParitionColumn(), + options.getPartitioningScheme(), + options.getFileFormat().getFileSuffix(), + options.getWriteDisposition(), + existingFiles, + FILTERED_RECORDS_OUT, + EXISTING_TARGET_FILES_OUT)); + + filteredRecordsTuple + .get(EXISTING_TARGET_FILES_OUT) + // Getting unique filenames + .apply(Distinct.create()) + .apply( + "Log existing target file names", + ParDo.of( + // This transform logs the distinct existing target files. Logging is done in + // a separate transform to prevent redundant logs. + // OutputT is String here since DoFn will not accept void. The resulting + // PCollection will be empty. + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + String filename = c.element(); + LOG.info( + "Target File {} already exists in the output asset bucket {}. Performing " + + " {} writeDisposition strategy.", + filename, + targetRootPath, + options.getWriteDisposition()); + } + })); + return filteredRecordsTuple.get(FILTERED_RECORDS_OUT); + } + @VisibleForTesting static void buildGcsPipeline( Pipeline pipeline, DataplexJdbcIngestionOptions options, DynamicDataSourceConfiguration dataSourceConfig, String targetRootPath) { - + List existingFiles = StorageUtils.getFilesInDirectory(targetRootPath); // Auto inferring beam schema Schema beamSchema = Schemas.jdbcSchemaToBeamSchema(dataSourceConfig.buildDatasource(), options.getQuery()); @@ -160,6 +261,19 @@ static void buildGcsPipeline( resultRows .apply("convert to GenericRecord", ParDo.of(new BeamRowToGenericRecordFn(avroSchema))) .setCoder(AvroCoder.of(avroSchema)); + + // If targetRootPath is changed in the following lines, please also change the root path for + // existingFiles + if (options.getParitionColumn() == null || options.getPartitioningScheme() == null) { + if (shouldSkipUnpartitionedTable(options, targetRootPath, existingFiles)) { + return; + } + } else { + genericRecords = + applyPartitionedWriteDispositionFilter( + genericRecords, options, targetRootPath, avroSchema, existingFiles); + } + // Write to GCS bucket PCollection metadata = genericRecords.apply( @@ -190,7 +304,10 @@ static void buildBigQueryPipeline( BigQueryIO.writeTableRows() .withoutValidation() .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_NEVER) - .withWriteDisposition(options.getWriteDisposition()) + // Mapping DataplexJdbcIngestionWriteDisposition.WriteDispositionOptions to + // BigqueryIO.Write.WriteDisposition + .withWriteDisposition( + MapWriteDisposition.mapWriteDispostion(options.getWriteDisposition())) .to(options.getOutputTable())); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java index af36f7f4c9..9f8eb76276 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitioned.java @@ -18,21 +18,17 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import com.google.cloud.teleport.v2.io.AvroSinkWithJodaDatesConversion; +import com.google.cloud.teleport.v2.utils.DataplexJdbcIngestionNaming; +import com.google.cloud.teleport.v2.utils.DataplexJdbcPartitionUtils; +import com.google.cloud.teleport.v2.utils.DataplexJdbcPartitionUtils.PartitioningSchema; import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.utils.SchemaUtils; import com.google.cloud.teleport.v2.values.PartitionMetadata; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import java.time.Instant; import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.ZonedDateTime; import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; import javax.annotation.Nullable; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.coders.KvCoder; @@ -41,7 +37,6 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.FileIO; import org.apache.beam.sdk.io.FileIO.Sink; -import org.apache.beam.sdk.io.FileIO.Write; import org.apache.beam.sdk.io.parquet.ParquetIO; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; @@ -50,20 +45,13 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; -import org.joda.time.ReadableInstant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * A {@link PTransform} that partitions a collection of {@link GenericRecord} by datetime field and - * writes the partitions to GCS. THe transform outputs a collection of {@link PartitionMetadata} for + * writes the partitions to GCS. The transform outputs a collection of {@link PartitionMetadata} for * each partition. - * - *

Three levels of partitioning granularity are supported by providing {@link - * PartitioningSchema}. - * - *

Avro logical type "timestamp-millis" is supported for partitioning, see: Logical types. */ public class GenericRecordsToGcsPartitioned extends PTransform, PCollection> { @@ -71,40 +59,6 @@ public class GenericRecordsToGcsPartitioned /* Logger for class.*/ private static final Logger LOG = LoggerFactory.getLogger(GenericRecordsToGcsPartitioned.class); - private static final ImmutableMap AVRO_DATE_TIME_LOGICAL_TYPES = - ImmutableMap.of( - LogicalTypes.timestampMillis(), ZoneOffset.UTC - // TODO(olegsa) add "local-timestamp-millis" to ZoneId.systemDefault() mapping when Avro - // version is updated - ); - - /** The granularity of partitioning. */ - public enum PartitioningSchema { - MONTHLY("month", ZonedDateTime::getMonthValue), - DAILY("day", ZonedDateTime::getDayOfMonth), - HOURLY("hour", ZonedDateTime::getHour); - - private final String label; - private final Function dateTimeToPartition; - - PartitioningSchema(String label, Function dateTimeToPartition) { - this.label = label; - this.dateTimeToPartition = dateTimeToPartition; - } - - private List> toPartition(ZonedDateTime dateTime) { - ImmutableList.Builder> result = ImmutableList.builder(); - result.add(KV.of("year", dateTime.getYear())); - for (PartitioningSchema schema : PartitioningSchema.values()) { - result.add(KV.of(schema.label, schema.dateTimeToPartition.apply(dateTime))); - if (this == schema) { - break; - } - } - return result.build(); - } - } - private final String gcsPath; private final String serializedAvroSchema; @@ -152,9 +106,10 @@ public PCollection expand(PCollection input) { .apply( "Write to Storage with No Partition", FileIO.write() - .withSuffix(outputFileFormat.getFileSuffix()) .via(sink) - .to(gcsPath)) + .to(gcsPath) + .withNaming(new DataplexJdbcIngestionNaming(outputFileFormat.getFileSuffix())) + .withNumShards(1)) // must be 1 as we can only have 1 file // Dummy conversion to Dataplex partition metadata // TODO(weiwenxu) Change after Dataplex metadata update is enabled .getPerDestinationOutputFilenames() @@ -175,7 +130,7 @@ public PartitionMetadata apply(String path) { })); } - ZoneId zoneId = getZoneId(schema); + ZoneId zoneId = DataplexJdbcPartitionUtils.getZoneId(schema, partitionColumnName); return input // partition and write files to GCS @@ -186,7 +141,8 @@ public PartitionMetadata apply(String path) { (GenericRecord r) -> partitioningSchema.toPartition( Instant.ofEpochMilli( - partitionColumnValueToMillis(r.get(partitionColumnName))) + DataplexJdbcPartitionUtils.partitionColumnValueToMillis( + r.get(partitionColumnName))) .atZone(zoneId))) // set the coder for the partition -- List> .withDestinationCoder( @@ -196,7 +152,10 @@ public PartitionMetadata apply(String path) { .withNumShards(1) // must be 1 as we can only have 1 file per Dataplex partition // derive filenames from the partition and output file format .withNaming( - p -> Write.defaultNaming(partitionToPath(p), outputFileFormat.getFileSuffix()))) + p -> + new DataplexJdbcIngestionNaming( + DataplexJdbcPartitionUtils.partitionToPath(p), + outputFileFormat.getFileSuffix()))) // convert the WriteFilesResult>> to Dataplex partition metadata .getPerDestinationOutputFilenames() .apply( @@ -224,68 +183,6 @@ public PartitionMetadata apply( })); } - private ZoneId getZoneId(Schema schema) { - Schema partitionFieldType = schema.getField(partitionColumnName).schema(); - // check if the partition field is nullable, inspired by {@code Schema.isNullable()} of Avro 1.9 - if (schema.getType() == Schema.Type.UNION) { - partitionFieldType = - partitionFieldType.getTypes().stream() - .filter(t -> t.getType() != Schema.Type.NULL) - .findFirst() - .orElseThrow( - () -> - new IllegalArgumentException( - String.format( - "Partition field %s is of unsupported type: %s", - partitionColumnName, schema.getField(partitionColumnName).schema()))); - } - - // get zone according to the logical-type if there is no logical-type assume UTC time-zone - ZoneId zoneId = - AVRO_DATE_TIME_LOGICAL_TYPES.getOrDefault( - partitionFieldType.getLogicalType(), ZoneOffset.UTC); - if (zoneId == null) { - throw new IllegalArgumentException( - String.format( - "Partition field `%s` is of an unsupported type: %s, supported types are `long` types" - + " with logical types: %s", - partitionColumnName, - partitionFieldType, - AVRO_DATE_TIME_LOGICAL_TYPES.keySet().stream() - .map(LogicalType::getName) - .collect(Collectors.joining(", ")))); - } - return zoneId; - } - - /** - * This method is used to address the static initialization in - * org.apache.beam.sdk.schemas.utils.AvroUtils static initialization. - * - *

A usage of AvroUtils changes how Avro treats `timestamp-millis` "globally", and so if - * AvroUtils is used, even in a unrelated classes, the `timestamp-millis` is returned as Joda - * timestamps, and if AvroUtils is not used `timestamp-millis` is returned as long. This method - * handles both cases and returns long millis. - */ - private static long partitionColumnValueToMillis(Object value) { - if (value instanceof Long) { - return (Long) value; - } else if (value instanceof ReadableInstant) { - return ((ReadableInstant) value).getMillis(); - } else { - throw new IllegalArgumentException( - "The partition column value is an instance of unsupported class: " + value.getClass()); - } - } - - private static String partitionToPath(List> partition) { - StringBuilder result = new StringBuilder(64); - for (KV element : partition) { - result.append(element.getKey()).append('=').append(element.getValue()).append('/'); - } - return result.toString(); - } - private static String withoutFileName(String gcsPath) { return gcsPath.substring(0, gcsPath.lastIndexOf('/')); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilter.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilter.java new file mode 100644 index 0000000000..3de935a22d --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilter.java @@ -0,0 +1,132 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +import com.google.cloud.teleport.v2.utils.DataplexJdbcPartitionUtils.PartitioningSchema; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.WriteDispositionOptions; +import java.time.Instant; +import java.time.ZoneId; +import java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.ProcessContext; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Filter to exclude unwanted records if the target file exists. */ +public class DataplexJdbcIngestionFilter + extends PTransform, PCollectionTuple> { + private static final Logger LOG = LoggerFactory.getLogger(DataplexJdbcIngestionFilter.class); + + private final String targetRootPath; + private final String partitionColumnName; + private final PartitioningSchema partitioningSchema; + private final String fileSuffix; + private final WriteDispositionOptions writeDisposition; + private final List existingFiles; + /** The tag for filtered records. */ + private TupleTag filteredRecordsOutputTag; + /** The tag for existing target file names. */ + private TupleTag existingTargetFilesOutputTag; + private ZoneId zoneId; + + public DataplexJdbcIngestionFilter( + String targetRootPath, + String serializedAvroSchema, + String partitionColumnName, + PartitioningSchema partitioningSchema, + String fileSuffix, + WriteDispositionOptions writeDisposition, + List existingFiles, + TupleTag filteredRecordsOutputTag, + TupleTag existingTargetFilesOutputTag) { + this.targetRootPath = targetRootPath; + this.partitionColumnName = partitionColumnName; + this.partitioningSchema = partitioningSchema; + this.fileSuffix = fileSuffix; + this.writeDisposition = writeDisposition; + this.existingFiles = existingFiles; + this.filteredRecordsOutputTag = filteredRecordsOutputTag; + this.existingTargetFilesOutputTag = existingTargetFilesOutputTag; + this.zoneId = + DataplexJdbcPartitionUtils.getZoneId( + SchemaUtils.parseAvroSchema(serializedAvroSchema), partitionColumnName); + } + + @Override + public PCollectionTuple expand(PCollection input) { + return input.apply( + ParDo.of(new DataplexJdbcIngestionFilterDoFn()) + .withOutputTags( + filteredRecordsOutputTag, TupleTagList.of(existingTargetFilesOutputTag))); + } + + private class DataplexJdbcIngestionFilterDoFn extends DoFn { + + private boolean shouldSkipRecord(String expectedFilePath) { + switch (writeDisposition) { + case WRITE_EMPTY: + throw new WriteDispositionException( + String.format( + "Target File %s already exists in the output asset bucket %s. Failing" + + " according to writeDisposition.", + expectedFilePath, targetRootPath)); + case SKIP: + return true; + case WRITE_TRUNCATE: + return false; + default: + throw new UnsupportedOperationException( + writeDisposition + " writeDisposition not implemented for writing to GCS."); + } + } + + @ProcessElement + public void processElement(ProcessContext c) { + GenericRecord record = c.element(); + List> partition = + partitioningSchema.toPartition( + Instant.ofEpochMilli( + DataplexJdbcPartitionUtils.partitionColumnValueToMillis( + record.get(partitionColumnName))) + .atZone(zoneId)); + String expectedFilePath = + new DataplexJdbcIngestionNaming( + DataplexJdbcPartitionUtils.partitionToPath(partition), fileSuffix) + .getSingleFilename(); + if (existingFiles.contains(expectedFilePath)) { + // Target file exists, performing writeDisposition strategy + if (!shouldSkipRecord(expectedFilePath)) { + c.output(record); + } + // Returning existing file name for logging + c.output(existingTargetFilesOutputTag, expectedFilePath); + } else { + // If target file does not exist, do not filter out the record + c.output(record); + } + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionNaming.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionNaming.java new file mode 100644 index 0000000000..421c53db35 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionNaming.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +import org.apache.beam.sdk.io.Compression; +import org.apache.beam.sdk.io.FileIO.Write; +import org.apache.beam.sdk.io.FileIO.Write.FileNaming; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; + +/** A FileNaming that generates filenames using Write.defaultNaming. */ +public class DataplexJdbcIngestionNaming implements FileNaming { + + private final FileNaming defaultNaming; + + public DataplexJdbcIngestionNaming(String prefix, String suffix) { + defaultNaming = Write.defaultNaming(prefix + "output", suffix); + } + + public DataplexJdbcIngestionNaming(String suffix) { + this("", suffix); + } + + public String getSingleFilename() { + return getFilename(GlobalWindow.INSTANCE, PaneInfo.NO_FIRING, 1, 0, Compression.AUTO); + } + + @Override + public String getFilename( + BoundedWindow window, PaneInfo pane, int numShards, int shardIndex, Compression compression) { + return defaultNaming.getFilename(window, pane, numShards, shardIndex, compression); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcPartitionUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcPartitionUtils.java new file mode 100644 index 0000000000..afde3dda74 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/DataplexJdbcPartitionUtils.java @@ -0,0 +1,141 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.beam.sdk.values.KV; +import org.joda.time.ReadableInstant; + +/** + * Utility functions for Partitions in DataplexJdbcIngestion template. + * + *

Avro logical type "timestamp-millis" is supported for partitioning, see: Logical types. + */ +public class DataplexJdbcPartitionUtils { + + private static final ImmutableMap AVRO_DATE_TIME_LOGICAL_TYPES = + ImmutableMap.of( + LogicalTypes.timestampMillis(), ZoneOffset.UTC + // TODO(olegsa) add "local-timestamp-millis" to ZoneId.systemDefault() mapping when Avro + // version is updated + ); + + /** + * The granularity of partitioning. + * + *

Three levels of partitioning granularity are supported by providing {@link + * PartitioningSchema}. + */ + public enum PartitioningSchema { + MONTHLY("month", ZonedDateTime::getMonthValue), + DAILY("day", ZonedDateTime::getDayOfMonth), + HOURLY("hour", ZonedDateTime::getHour); + + private final String label; + private final Function dateTimeToPartition; + + PartitioningSchema(String label, Function dateTimeToPartition) { + this.label = label; + this.dateTimeToPartition = dateTimeToPartition; + } + + public List> toPartition(ZonedDateTime dateTime) { + ImmutableList.Builder> result = ImmutableList.builder(); + result.add(KV.of("year", dateTime.getYear())); + for (PartitioningSchema schema : PartitioningSchema.values()) { + result.add(KV.of(schema.label, schema.dateTimeToPartition.apply(dateTime))); + if (this == schema) { + break; + } + } + return result.build(); + } + } + + public static ZoneId getZoneId(Schema schema, String partitionColumnName) { + Schema partitionFieldType = schema.getField(partitionColumnName).schema(); + // check if the partition field is nullable, inspired by {@code Schema.isNullable()} of Avro 1.9 + if (schema.getType() == Schema.Type.UNION) { + partitionFieldType = + partitionFieldType.getTypes().stream() + .filter(t -> t.getType() != Schema.Type.NULL) + .findFirst() + .orElseThrow( + () -> + new IllegalArgumentException( + String.format( + "Partition field %s is of unsupported type: %s", + partitionColumnName, schema.getField(partitionColumnName).schema()))); + } + + // get zone according to the logical-type if there is no logical-type assume UTC time-zone + ZoneId zoneId = + AVRO_DATE_TIME_LOGICAL_TYPES.getOrDefault( + partitionFieldType.getLogicalType(), ZoneOffset.UTC); + if (zoneId == null) { + throw new IllegalArgumentException( + String.format( + "Partition field `%s` is of an unsupported type: %s, supported types are `long` types" + + " with logical types: %s", + partitionColumnName, + partitionFieldType, + AVRO_DATE_TIME_LOGICAL_TYPES.keySet().stream() + .map(LogicalType::getName) + .collect(Collectors.joining(", ")))); + } + return zoneId; + } + + /** + * This method is used to address the static initialization in + * org.apache.beam.sdk.schemas.utils.AvroUtils static initialization. + * + *

A usage of AvroUtils changes how Avro treats `timestamp-millis` "globally", and so if + * AvroUtils is used, even in a unrelated classes, the `timestamp-millis` is returned as Joda + * timestamps, and if AvroUtils is not used `timestamp-millis` is returned as long. This method + * handles both cases and returns long millis. + */ + public static long partitionColumnValueToMillis(Object value) { + if (value instanceof Long) { + return (Long) value; + } else if (value instanceof ReadableInstant) { + return ((ReadableInstant) value).getMillis(); + } else { + throw new IllegalArgumentException( + "The partition column value is an instance of unsupported class: " + value.getClass()); + } + } + + /** Generates path from partition value. */ + public static String partitionToPath(List> partition) { + StringBuilder result = new StringBuilder(64); + for (KV element : partition) { + result.append(element.getKey()).append('=').append(element.getValue()).append('/'); + } + return result.toString(); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java index 67936d3845..1c73630f67 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcConverters.java @@ -26,7 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** Common code for Teleport DataplexJdbcIngestion. */ +/** Common code for Jdbc templates. */ public class JdbcConverters { private static final Logger LOG = LoggerFactory.getLogger(JdbcConverters.class); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcIngestionWriteDisposition.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcIngestionWriteDisposition.java new file mode 100644 index 0000000000..b630b05877 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/JdbcIngestionWriteDisposition.java @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +import java.util.EnumMap; +import java.util.Map; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; + +/** + * Exposes WriteDispositionOptions, WriteDispositionException and MapWriteDisposition for + * DataplexJdbcIngestion template. + */ +public class JdbcIngestionWriteDisposition { + + /** + * Provides the possible WriteDispositionOptions when writing to BigQuery/GCS and target exists. + */ + public enum WriteDispositionOptions { + WRITE_APPEND("WRITE_APPEND"), + WRITE_TRUNCATE("WRITE_TRUNCATE"), + WRITE_EMPTY("WRITE_EMPTY"), + SKIP("SKIP"); + + private final String writeDispositionOption; + + WriteDispositionOptions(String writeDispositionOption) { + this.writeDispositionOption = writeDispositionOption; + } + } + + /** + * Thrown if {@link + * com.google.cloud.teleport.v2.utils.DataplexJdbcIngestionWriteDisposition.WriteDispositionOptions + * WriteDispositionOptions} is set to {@code WRITE_EMPTY} and a target file exists. + */ + public static class WriteDispositionException extends RuntimeException { + public WriteDispositionException(String message) { + super(message); + } + } + + /** Maps WriteDispostionOptions to BigQueryIO.Write.WriteDisposition. */ + public static class MapWriteDisposition { + private static final Map WRITE_DISPOSITION_MAP; + + static { + WRITE_DISPOSITION_MAP = new EnumMap<>(WriteDispositionOptions.class); + WRITE_DISPOSITION_MAP.put( + WriteDispositionOptions.WRITE_APPEND, WriteDisposition.WRITE_APPEND); + WRITE_DISPOSITION_MAP.put( + WriteDispositionOptions.WRITE_TRUNCATE, WriteDisposition.WRITE_TRUNCATE); + WRITE_DISPOSITION_MAP.put(WriteDispositionOptions.WRITE_EMPTY, WriteDisposition.WRITE_EMPTY); + } + + public static WriteDisposition mapWriteDispostion(WriteDispositionOptions option) { + if (option == WriteDispositionOptions.SKIP) { + throw new UnsupportedOperationException( + "SKIP WriteDisposition not implemented for writing to BigQuery."); + } + return WRITE_DISPOSITION_MAP.get(option); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java index 3efeb678dc..7db5275ddb 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/GenericRecordsToGcsPartitionedTest.java @@ -15,7 +15,7 @@ */ package com.google.cloud.teleport.v2.transforms; -import com.google.cloud.teleport.v2.transforms.GenericRecordsToGcsPartitioned.PartitioningSchema; +import com.google.cloud.teleport.v2.utils.DataplexJdbcPartitionUtils.PartitioningSchema; import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; import com.google.cloud.teleport.v2.values.PartitionMetadata; import com.google.common.collect.ImmutableList; diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilterTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilterTest.java new file mode 100644 index 0000000000..50eea35861 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/utils/DataplexJdbcIngestionFilterTest.java @@ -0,0 +1,179 @@ +/* + * Copyright (C) 2022 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.v2.utils; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +import com.google.cloud.teleport.v2.utils.DataplexJdbcPartitionUtils.PartitioningSchema; +import com.google.cloud.teleport.v2.utils.FileFormat.FileFormatOptions; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.WriteDispositionException; +import com.google.cloud.teleport.v2.utils.JdbcIngestionWriteDisposition.WriteDispositionOptions; +import java.io.File; +import java.io.IOException; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Parser; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link DataplexJdbcIngestionFilter}. */ +@RunWith(JUnit4.class) +public final class DataplexJdbcIngestionFilterTest { + private static final String SERIALIZED_SCHEMA_TEMPLATE = + "{" + + "\"name\": \"Schema\"," + + "\"type\": \"record\"," + + "\"fields\": [" + + " {\"name\": \"x\", \"type\": \"boolean\"}," + + " {\"name\": \"date\", \"type\": {\"type\": \"%s\", \"logicalType\": \"%s\"}}" + + "]" + + "}"; + private static final String SERIALIZED_SCHEMA = + String.format(SERIALIZED_SCHEMA_TEMPLATE, "long", "timestamp-millis"); + private static final Schema SCHEMA = new Parser().parse(SERIALIZED_SCHEMA); + private static final String PARTITION_COLUMN_NAME = "date"; + private static final TupleTag FILTERED_RECORDS_OUT = + new TupleTag() {}; + private static final TupleTag EXISTING_TARGET_FILES_OUT = new TupleTag() {}; + private Record record11; + private Record record12; + private Record record21; + + @Rule public final transient TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule public final transient TestPipeline mainPipeline = TestPipeline.create(); + + @Before + public void setUp() throws IOException { + record11 = new Record(SCHEMA); + record11.put("x", true); + record11.put("date", dateToEpochMillis(2010, 1)); + record12 = new Record(SCHEMA); + record12.put("x", false); + record12.put("date", dateToEpochMillis(2010, 1)); + record21 = new Record(SCHEMA); + record21.put("x", true); + record21.put("date", dateToEpochMillis(2010, 2)); + File outputDir1 = temporaryFolder.newFolder("year=2010", "month=1"); + File outputFile1 = new File(outputDir1.getAbsolutePath() + "/" + "output-00000-of-00001.avro"); + outputFile1.createNewFile(); + } + + @Test + public void testRecordOverwriteIfTargetFileExists() { + String targetRootPath = temporaryFolder.getRoot().getAbsolutePath(); + PCollectionTuple result = + mainPipeline + .apply( + Create.of(record11, record12, record21) + .withCoder(AvroCoder.of(SCHEMA))) + .apply( + new DataplexJdbcIngestionFilter( + targetRootPath, + SERIALIZED_SCHEMA, + PARTITION_COLUMN_NAME, + PartitioningSchema.MONTHLY, + FileFormatOptions.AVRO.getFileSuffix(), + WriteDispositionOptions.WRITE_TRUNCATE, + StorageUtils.getFilesInDirectory(targetRootPath), + FILTERED_RECORDS_OUT, + EXISTING_TARGET_FILES_OUT)); + PAssert.that(result.get(FILTERED_RECORDS_OUT)).containsInAnyOrder(record11, record12, record21); + // Contains the same filename two times since 2 records map to the same partition whose file + // exists + PAssert.that(result.get(EXISTING_TARGET_FILES_OUT)) + .containsInAnyOrder( + "year=2010/month=1/output-00000-of-00001.avro", + "year=2010/month=1/output-00000-of-00001.avro"); + mainPipeline.run(); + } + + @Test + public void testRecordSkippedIfTargetFileExists() { + String targetRootPath = temporaryFolder.getRoot().getAbsolutePath(); + PCollectionTuple result = + mainPipeline + .apply( + Create.of(record11, record12, record21) + .withCoder(AvroCoder.of(SCHEMA))) + .apply( + new DataplexJdbcIngestionFilter( + targetRootPath, + SERIALIZED_SCHEMA, + PARTITION_COLUMN_NAME, + PartitioningSchema.MONTHLY, + FileFormatOptions.AVRO.getFileSuffix(), + WriteDispositionOptions.SKIP, + StorageUtils.getFilesInDirectory(targetRootPath), + FILTERED_RECORDS_OUT, + EXISTING_TARGET_FILES_OUT)); + PAssert.that(result.get(FILTERED_RECORDS_OUT)).containsInAnyOrder(record21); + // Contains the same filename two times since 2 records map to the same partition whose file + // exists + PAssert.that(result.get(EXISTING_TARGET_FILES_OUT)) + .containsInAnyOrder( + "year=2010/month=1/output-00000-of-00001.avro", + "year=2010/month=1/output-00000-of-00001.avro"); + mainPipeline.run(); + } + + @Test + public void testFailIfTargetFileExists() { + String targetRootPath = temporaryFolder.getRoot().getAbsolutePath(); + PCollectionTuple result = + mainPipeline + .apply( + Create.of(record11, record12, record21) + .withCoder(AvroCoder.of(SCHEMA))) + .apply( + new DataplexJdbcIngestionFilter( + targetRootPath, + SERIALIZED_SCHEMA, + PARTITION_COLUMN_NAME, + PartitioningSchema.MONTHLY, + FileFormatOptions.AVRO.getFileSuffix(), + WriteDispositionOptions.WRITE_EMPTY, + StorageUtils.getFilesInDirectory(targetRootPath), + FILTERED_RECORDS_OUT, + EXISTING_TARGET_FILES_OUT)); + try { + mainPipeline.run(); + fail("Expected a WriteDispositionException."); + } catch (Exception e) { + assertThat(e).hasCauseThat().isInstanceOf(WriteDispositionException.class); + } + } + + private static long dateToEpochMillis(int year, int month) { + return ZonedDateTime.of(year, month, 1, 1, 42, 42, 42, ZoneOffset.UTC) + .toInstant() + .toEpochMilli(); + } +} From 942f172f62c01b34a439d6deeda81d0bf743a51f Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 4 Apr 2022 17:14:33 -0700 Subject: [PATCH 096/145] Add support for building change streams DDL from information schema in Cloud Spanner Import/Export pipelines. PiperOrigin-RevId: 439448719 --- .../teleport/spanner/ddl/ChangeStream.java | 94 +++++++++++++++ .../cloud/teleport/spanner/ddl/Ddl.java | 50 ++++++++ .../spanner/ddl/InformationSchemaScanner.java | 110 +++++++++++++++++- .../cloud/teleport/spanner/ddl/DdlTest.java | 27 +++++ .../ddl/InformationSchemaScannerTest.java | 27 +++++ 5 files changed, 307 insertions(+), 1 deletion(-) create mode 100644 src/main/java/com/google/cloud/teleport/spanner/ddl/ChangeStream.java diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/ChangeStream.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/ChangeStream.java new file mode 100644 index 0000000000..dc3e09ce3d --- /dev/null +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/ChangeStream.java @@ -0,0 +1,94 @@ +/* + * Copyright (C) 2022 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.ddl; + +import com.google.auto.value.AutoValue; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.Serializable; +import javax.annotation.Nullable; + +/** Cloud Spanner change stream. */ +@AutoValue +public abstract class ChangeStream implements Serializable { + + private static final long serialVersionUID = 1L; + + public abstract String name(); + + @Nullable + public abstract String forClause(); + + @Nullable + public abstract ImmutableList options(); + + public abstract Builder toBuilder(); + + public static Builder builder() { + return new AutoValue_ChangeStream.Builder(); + } + + public void prettyPrint(Appendable appendable) throws IOException { + appendable.append("CREATE CHANGE STREAM `").append(name()).append("`"); + if (!Strings.isNullOrEmpty(forClause())) { + appendable.append("\n\t").append(forClause()); + } + if (options() != null && !options().isEmpty()) { + String optionsString = String.join(", ", options()); + appendable.append("\n\t").append("OPTIONS (").append(optionsString).append(")"); + } + } + + public String prettyPrint() { + StringBuilder sb = new StringBuilder(); + try { + prettyPrint(sb); + } catch (IOException e) { + throw new RuntimeException(e); + } + return sb.toString(); + } + + @Override + public String toString() { + return prettyPrint(); + } + + /** A builder for {@link ChangeStream}. */ + @AutoValue.Builder + public abstract static class Builder { + private Ddl.Builder ddlBuilder; + + public Builder ddlBuilder(Ddl.Builder ddlBuilder) { + this.ddlBuilder = ddlBuilder; + return this; + } + + public abstract Builder name(String name); + + public abstract Builder forClause(String name); + + public abstract Builder options(ImmutableList options); + + public abstract ChangeStream build(); + + public Ddl.Builder endChangeStream() { + ddlBuilder.addChangeStream(build()); + return ddlBuilder; + } + } +} diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java index c7c693a8f9..3437f89947 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java @@ -45,6 +45,7 @@ public class Ddl implements Serializable { private ImmutableSortedMap tables; private ImmutableSortedMap views; + private ImmutableSortedMap changeStreams; private TreeMultimap parents; // This is only populated by InformationSchemaScanner and not while reading from AVRO files. private TreeMultimap referencedTables; @@ -53,11 +54,13 @@ public class Ddl implements Serializable { private Ddl( ImmutableSortedMap tables, ImmutableSortedMap views, + ImmutableSortedMap changeStreams, TreeMultimap parents, TreeMultimap referencedTables, ImmutableList databaseOptions) { this.tables = tables; this.views = views; + this.changeStreams = changeStreams; this.parents = parents; this.referencedTables = referencedTables; this.databaseOptions = databaseOptions; @@ -121,6 +124,14 @@ public View view(String viewName) { return views.get(viewName.toLowerCase()); } + public Collection changeStreams() { + return changeStreams.values(); + } + + public ChangeStream changeStream(String changeStreamName) { + return changeStreams.get(changeStreamName.toLowerCase()); + } + public ImmutableList databaseOptions() { return databaseOptions; } @@ -161,6 +172,11 @@ public void prettyPrint(Appendable appendable) throws IOException { appendable.append("\n"); view.prettyPrint(appendable); } + + for (ChangeStream changeStream : changeStreams()) { + appendable.append("\n"); + changeStream.prettyPrint(appendable); + } } public List statements() { @@ -169,6 +185,7 @@ public List statements() { .addAll(createIndexStatements()) .addAll(addForeignKeyStatements()) .addAll(createViewStatements()) + .addAll(createChangeStreamStatements()) .addAll(setOptionsStatements("%db_name%")) .build(); } @@ -227,6 +244,14 @@ public List createViewStatements() { return result; } + public List createChangeStreamStatements() { + List result = new ArrayList<>(changeStreams.size()); + for (ChangeStream changeStream : changeStreams()) { + result.add(changeStream.prettyPrint()); + } + return result; + } + public List setOptionsStatements(String databaseId) { List result = new ArrayList<>(); for (Export.DatabaseOption databaseOption : databaseOptions()) { @@ -289,6 +314,7 @@ public static class Builder { private Map tables = Maps.newLinkedHashMap(); private Map views = Maps.newLinkedHashMap(); + private Map changeStreams = Maps.newLinkedHashMap(); private TreeMultimap parents = TreeMultimap.create(); private TreeMultimap referencedTables = TreeMultimap.create(); private ImmutableList databaseOptions = ImmutableList.of(); @@ -333,6 +359,22 @@ public boolean hasView(String name) { return views.containsKey(name.toLowerCase()); } + public ChangeStream.Builder createChangeStream(String name) { + ChangeStream changeStream = changeStreams.get(name.toLowerCase()); + if (changeStream == null) { + return ChangeStream.builder().name(name).ddlBuilder(this); + } + return changeStream.toBuilder().ddlBuilder(this); + } + + public void addChangeStream(ChangeStream changeStream) { + changeStreams.put(changeStream.name().toLowerCase(), changeStream); + } + + public boolean hasChangeStream(String name) { + return changeStreams.containsKey(name.toLowerCase()); + } + public void mergeDatabaseOptions(List databaseOptions) { List allowedDatabaseOptions = new ArrayList<>(); List existingOptionNames = new ArrayList<>(); @@ -356,6 +398,7 @@ public Ddl build() { return new Ddl( ImmutableSortedMap.copyOf(tables), ImmutableSortedMap.copyOf(views), + ImmutableSortedMap.copyOf(changeStreams), parents, referencedTables, databaseOptions); @@ -366,6 +409,7 @@ public Builder toBuilder() { Builder builder = new Builder(); builder.tables.putAll(tables); builder.views.putAll(views); + builder.changeStreams.putAll(changeStreams); builder.parents.putAll(parents); builder.referencedTables.putAll(referencedTables); builder.databaseOptions = databaseOptions; @@ -397,6 +441,11 @@ public boolean equals(Object o) { if (views != null ? !views.equals(ddl.views) : ddl.views != null) { return false; } + if (changeStreams != null + ? !changeStreams.equals(ddl.changeStreams) + : ddl.changeStreams != null) { + return false; + } return databaseOptions.equals(ddl.databaseOptions); } @@ -406,6 +455,7 @@ public int hashCode() { result = 31 * result + (parents != null ? parents.hashCode() : 0); result = 31 * result + (referencedTables != null ? referencedTables.hashCode() : 0); result = 31 * result + (views != null ? views.hashCode() : 0); + result = 31 * result + (changeStreams != null ? changeStreams.hashCode() : 0); result = 31 * result + (databaseOptions != null ? databaseOptions.hashCode() : 0); return result; } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java index 9f15bd99f2..e562171997 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java @@ -22,8 +22,10 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; +import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.stream.Collectors; import org.apache.beam.sdk.values.KV; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,6 +48,8 @@ public Ddl scan() { listViews(builder); listColumns(builder); listColumnOptions(builder); + listChangeStreams(builder); + listChangeStreamOptions(builder); Map> indexes = Maps.newHashMap(); listIndexes(indexes); listIndexColumns(builder, indexes); @@ -293,7 +297,8 @@ private void listColumnOptions(Ddl.Builder builder) { ResultSet resultSet = context.executeQuery( Statement.newBuilder( - "SELECT t.table_name, t.column_name, t.option_name, t.option_type, t.option_value " + "SELECT t.table_name, t.column_name," + + " t.option_name, t.option_type, t.option_value " + " FROM information_schema.column_options AS t " + " WHERE t.table_catalog = '' AND t.table_schema = ''" + " ORDER BY t.table_name, t.column_name") @@ -437,4 +442,107 @@ private void listViews(Ddl.Builder builder) { builder.createView(viewName).query(viewQuery).security(View.SqlSecurity.INVOKER).endView(); } } + + private void listChangeStreams(Ddl.Builder builder) { + ResultSet resultSet = + context.executeQuery( + Statement.of( + "SELECT cs.change_stream_name," + + " cs.all," + + " cst.table_name," + + " cst.all_columns," + + " ARRAY_AGG(csc.column_name IGNORE NULLS) AS column_name_list" + + " FROM information_schema.change_streams AS cs" + + " LEFT JOIN information_schema.change_stream_tables AS cst" + + " ON cs.change_stream_catalog = cst.change_stream_catalog" + + " AND cs.change_stream_schema = cst.change_stream_schema" + + " AND cs.change_stream_name = cst.change_stream_name" + + " LEFT JOIN information_schema.change_stream_columns AS csc" + + " ON cst.change_stream_catalog = csc.change_stream_catalog" + + " AND cst.change_stream_schema = csc.change_stream_schema" + + " AND cst.change_stream_name = csc.change_stream_name" + + " AND cst.table_catalog = csc.table_catalog" + + " AND cst.table_schema = csc.table_schema" + + " AND cst.table_name = csc.table_name" + + " WHERE cs.change_stream_catalog = ''" + + " AND cs.change_stream_schema=''" + + " GROUP BY cs.change_stream_name, cs.all, cst.table_name, cst.all_columns" + + " ORDER BY cs.change_stream_name, cs.all, cst.table_name")); + + Map allChangeStreams = Maps.newHashMap(); + while (resultSet.next()) { + String changeStreamName = resultSet.getString(0); + boolean all = resultSet.getBoolean(1); + String tableName = resultSet.isNull(2) ? null : resultSet.getString(2); + Boolean allColumns = resultSet.isNull(3) ? null : resultSet.getBoolean(3); + List columnNameList = resultSet.isNull(4) ? null : resultSet.getStringList(4); + + StringBuilder forClause = + allChangeStreams.computeIfAbsent(changeStreamName, k -> new StringBuilder()); + if (all) { + forClause.append("FOR ALL"); + continue; + } else if (tableName == null) { + // The change stream does not track any table/column, i.e., it does not have a for-clause. + continue; + } + + forClause.append(forClause.length() == 0 ? "FOR " : ", "); + forClause.append("`").append(tableName).append("`"); + if (allColumns) { + continue; + } else if (columnNameList == null) { + forClause.append("()"); + } else { + String sortedColumns = + columnNameList.stream().sorted().collect(Collectors.joining("`, `", "(`", "`)")); + forClause.append(sortedColumns); + } + } + + for (Map.Entry entry : allChangeStreams.entrySet()) { + String changeStreamName = entry.getKey(); + StringBuilder forClause = entry.getValue(); + builder + .createChangeStream(changeStreamName) + .forClause(forClause.toString()) + .endChangeStream(); + } + } + + private void listChangeStreamOptions(Ddl.Builder builder) { + ResultSet resultSet = + context.executeQuery( + Statement.of( + "SELECT t.change_stream_name, t.option_name, t.option_type, t.option_value" + + " FROM information_schema.change_stream_options AS t" + + " WHERE t.change_stream_catalog = '' AND t.change_stream_schema = ''" + + " ORDER BY t.change_stream_name, t.option_name")); + + Map> allOptions = Maps.newHashMap(); + while (resultSet.next()) { + String changeStreamName = resultSet.getString(0); + String optionName = resultSet.getString(1); + String optionType = resultSet.getString(2); + String optionValue = resultSet.getString(3); + + ImmutableList.Builder options = + allOptions.computeIfAbsent(changeStreamName, k -> ImmutableList.builder()); + if (optionType.equalsIgnoreCase("STRING")) { + options.add( + optionName + + "=\"" + + DdlUtilityComponents.OPTION_STRING_ESCAPER.escape(optionValue) + + "\""); + } else { + options.add(optionName + "=" + optionValue); + } + } + + for (Map.Entry> entry : allOptions.entrySet()) { + String changeStreamName = entry.getKey(); + ImmutableList options = entry.getValue().build(); + builder.createChangeStream(changeStreamName).options(options).endChangeStream(); + } + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index 2dff205e5f..27f8953959 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -176,4 +176,31 @@ public void testDatabaseOptions() { optionStatements.get(0), is("ALTER DATABASE `database_id` SET OPTIONS ( version_retention_period = 4d )")); } + + @Test + public void changeStreams() { + Ddl ddl = + Ddl.builder() + .createChangeStream("ChangeStreamAll") + .forClause("FOR ALL") + .options( + ImmutableList.of( + "retention_period=\"7d\"", "value_capture_type=\"OLD_AND_NEW_VALUES\"")) + .endChangeStream() + .createChangeStream("ChangeStreamEmpty") + .endChangeStream() + .createChangeStream("ChangeStreamTableColumns") + .forClause("FOR `T1`, `T2`(`c1`, `c2`), `T3`()") + .endChangeStream() + .build(); + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE CHANGE STREAM `ChangeStreamAll`" + + " FOR ALL" + + " OPTIONS (retention_period=\"7d\", value_capture_type=\"OLD_AND_NEW_VALUES\")" + + " CREATE CHANGE STREAM `ChangeStreamEmpty`" + + " CREATE CHANGE STREAM `ChangeStreamTableColumns`" + + " FOR `T1`, `T2`(`c1`, `c2`), `T3`()")); + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java index a3ce233fdd..6aca2fb98a 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java @@ -360,4 +360,31 @@ public void databaseOptions() throws Exception { statements.set(0, alterStatement.replace(dbId, "%db_name%")); assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); } + + @Test + public void changeStreams() throws Exception { + List statements = + Arrays.asList( + "CREATE TABLE `Account` (" + + " `id` INT64 NOT NULL," + + " `balanceId` INT64 NOT NULL," + + " `balance` FLOAT64 NOT NULL," + + " ) PRIMARY KEY (`id` ASC)", + " CREATE TABLE `Users` (" + + " `id` INT64 NOT NULL," + + " `first_name` STRING(10)," + + " `last_name` STRING(MAX)," + + " `age` INT64," + + " ) PRIMARY KEY (`id` ASC)", + " CREATE CHANGE STREAM `ChangeStreamAll` FOR ALL" + + " OPTIONS (retention_period=\"7d\", value_capture_type=\"OLD_AND_NEW_VALUES\")", + " CREATE CHANGE STREAM `ChangeStreamEmpty`" + " OPTIONS (retention_period=\"24h\")", + " CREATE CHANGE STREAM `ChangeStreamKeyColumns` FOR `Account`(), `Users`()", + " CREATE CHANGE STREAM `ChangeStreamTableColumns`" + + " FOR `Account`, `Users`(`first_name`, `last_name`)"); + + spannerServer.createDatabase(dbId, statements); + Ddl ddl = getDatabaseDdl(); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); + } } From fe9be821326347ccdf3650a0683bd988699c749d Mon Sep 17 00:00:00 2001 From: olegsa Date: Thu, 7 Apr 2022 13:11:59 -0700 Subject: [PATCH 097/145] Dataplex Tiering template: fix for the handling of the TIME fields. The CL updates the BigQuery to GCS transform to handle the fields with the type TIME. There is misalignment between Apache Beam and BigQuery API when translating the TIME BigQuery type to Avro type. Beam translates it as `string` while BigQuery translates it as `int` with a "logicalType" `time-micros`. Also the TAP tests were timing out sometimes, successful runs were close to 900 seconds limit for large tests. PiperOrigin-RevId: 440182344 --- .../BigQueryTableToGcsTransform.java | 34 ++++++--- .../cloud/teleport/v2/utils/Schemas.java | 6 ++ .../templates/DataplexBigQueryToGcsTest.java | 74 +++++++++++++++---- 3 files changed, 88 insertions(+), 26 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java index 022b38472c..47f117af1d 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/BigQueryTableToGcsTransform.java @@ -24,6 +24,8 @@ import com.google.cloud.teleport.v2.values.DataplexCompression; import com.google.common.annotations.VisibleForTesting; import java.time.LocalDate; +import java.time.LocalTime; +import java.time.temporal.ChronoField; import java.util.ArrayList; import java.util.List; import org.apache.avro.LogicalTypes; @@ -229,24 +231,34 @@ private TypedRead getDefaultRead() { * To fix this mismatch this cmethod converts the `string` dates fields to `int` with logical type * `date` fields. * - *

Note that for the TIMESTAMP type both Beam's BigQueryIO and BQ API map it to `long` so there - * is no mismatch. + *

Note that for the TIMESTAMP type both Beam's BigQueryIO and BQ API map it to `long`,and for + * the DATETIME both map it to `string` so there is no mismatch for those types. */ private GenericRecord genericRecordWithFixedDates(SchemaAndRecord schemaAndRecord) { - GenericRecord input = schemaAndRecord.getRecord(); + return genericRecordWithFixedDates(schemaAndRecord.getRecord()); + } + + private GenericRecord genericRecordWithFixedDates(GenericRecord input) { GenericRecord output = new GenericData.Record(table.getSchema()); List fields = table.getSchema().getFields(); for (int i = 0; i < fields.size(); i++) { - if (Schemas.isSchemaOfTypeOrNullableType( - fields.get(i).schema(), Schema.Type.INT, LogicalTypes.date())) { - Object value = input.get(i); - if (!(value instanceof CharSequence)) { - throw new IllegalStateException( - "The class of input value of type DATE is " + value.getClass()); - } + Object value = input.get(i); + Schema fieldSchema = fields.get(i).schema(); + if (value == null) { + output.put(i, null); + } else if (Schemas.isSchemaOfTypeOrNullableType(fieldSchema, Schema.Type.RECORD) + && value instanceof GenericRecord) { + output.put(i, genericRecordWithFixedDates((GenericRecord) value)); + } else if (Schemas.isSchemaOfTypeOrNullableType( + fields.get(i).schema(), Schema.Type.INT, LogicalTypes.date()) + && value instanceof CharSequence) { output.put(i, (int) LocalDate.parse((CharSequence) value).toEpochDay()); + } else if (Schemas.isSchemaOfTypeOrNullableType( + fields.get(i).schema(), Schema.Type.LONG, LogicalTypes.timeMicros()) + && value instanceof CharSequence) { + output.put(i, LocalTime.parse((CharSequence) value).getLong(ChronoField.MICRO_OF_DAY)); } else { - output.put(i, input.get(i)); + output.put(i, value); } } return output; diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java index bd00b9f4da..8b57b6db9f 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/utils/Schemas.java @@ -89,6 +89,12 @@ public static boolean isSchemaOfTypeOrNullableType( .anyMatch(t -> isSchemaOfTypeOrNullableType(t, type, logicalType)); } + public static boolean isSchemaOfTypeOrNullableType(Schema schema, Schema.Type type) { + return Objects.equals(type, schema.getType()) + || Objects.equals(Schema.Type.UNION, schema.getType()) + && schema.getTypes().stream().anyMatch(s -> isSchemaOfTypeOrNullableType(s, type)); + } + private static Schema dataplexFieldsToAvro( List dataplexFields, RecordBuilder avroRecordBuilder) diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java index 77bf6ee0de..d94116f370 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/DataplexBigQueryToGcsTest.java @@ -149,6 +149,8 @@ public void setUp() throws InterruptedException, IOException { new TableFieldSchema().setName("ts").setType("TIMESTAMP"), new TableFieldSchema().setName("s1").setType("STRING"), new TableFieldSchema().setName("d1").setType("DATE"), + new TableFieldSchema().setName("t1").setType("TIME").setMode("REQUIRED"), + new TableFieldSchema().setName("dt").setType("DATETIME"), new TableFieldSchema().setName("i1").setType("INTEGER"))); avroSchema = @@ -158,6 +160,8 @@ public void setUp() throws InterruptedException, IOException { + "[{\"name\":\"ts\",\"type\":[\"null\",{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}]}," + "{\"name\":\"s1\",\"type\":[\"null\",\"string\"]}," + "{\"name\":\"d1\",\"type\":[\"null\",{\"type\":\"int\",\"logicalType\":\"date\"}]}," + + "{\"name\":\"t1\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}," + + "{\"name\":\"dt\",\"type\":[\"null\",{\"type\":\"string\",\"logicalType\":\"datetime\"}]}," + "{\"name\":\"i1\",\"type\":[\"null\",\"long\"]}]}"); long modTime = System.currentTimeMillis() * 1000; @@ -199,20 +203,55 @@ public void setUp() throws InterruptedException, IOException { defaultRecords = new TableRow[] { - new TableRow().set("ts", 1L).set("s1", "1001").set("d1", "1970-01-01").set("i1", 2001L), - new TableRow().set("ts", 2L).set("s1", "1002").set("d1", "1970-01-02").set("i1", 2002L), - new TableRow().set("ts", 3L).set("s1", "1003").set("d1", "1970-01-03").set("i1", 2003L), - new TableRow().set("ts", 4L).set("s1", "1004").set("d1", "1970-01-04").set("i1", null), - new TableRow().set("ts", 5L).set("s1", "1005").set("d1", "1970-01-05").set("i1", 2005L) + new TableRow() + .set("ts", 1L) + .set("s1", "1001") + .set("d1", "1970-01-01") + .set("t1", "00:00:00.000001") + .set("dt", "2020-01-01T00:42:00.123") + .set("i1", 2001L), + new TableRow() + .set("ts", 2L) + .set("s1", "1002") + .set("d1", "1970-01-02") + .set("t1", "00:00:00.000002") + .set("dt", "2020-01-02T00:42:00.123") + .set("i1", 2002L), + new TableRow() + .set("ts", 3L) + .set("s1", "1003") + .set("d1", "1970-01-03") + .set("t1", "00:00:00.000003") + .set("dt", "2020-01-03T00:42:00.123") + .set("i1", 2003L), + new TableRow() + .set("ts", 4L) + .set("s1", "1004") + .set("d1", "1970-01-04") + .set("t1", "00:00:00.000004") + .set("dt", "2020-01-04T00:42:00.123") + .set("i1", null), + new TableRow() + .set("ts", 5L) + .set("s1", "1005") + .set("d1", "1970-01-05") + .set("t1", "00:00:00.000005") + .set("dt", "2020-01-05T00:42:00.123") + .set("i1", 2005L) }; defaultExpectedRecords = new String[] { - "{\"ts\": 1, \"s1\": \"1001\", \"d1\": 0, \"i1\": 2001}", - "{\"ts\": 2, \"s1\": \"1002\", \"d1\": 1, \"i1\": 2002}", - "{\"ts\": 3, \"s1\": \"1003\", \"d1\": 2, \"i1\": 2003}", - "{\"ts\": 4, \"s1\": \"1004\", \"d1\": 3, \"i1\": null}", - "{\"ts\": 5, \"s1\": \"1005\", \"d1\": 4, \"i1\": 2005}" + "{\"ts\": 1, \"s1\": \"1001\", \"d1\": 0, \"t1\": 1, \"dt\": \"2020-01-01T00:42:00.123\"," + + " \"i1\": 2001}", + "{\"ts\": 2, \"s1\": \"1002\", \"d1\": 1, \"t1\": 2, \"dt\": \"2020-01-02T00:42:00.123\"," + + " \"i1\": 2002}", + "{\"ts\": 3, \"s1\": \"1003\", \"d1\": 2, \"t1\": 3, \"dt\": \"2020-01-03T00:42:00.123\"," + + " \"i1\": 2003}", + "{\"ts\": 4, \"s1\": \"1004\", \"d1\": 3, \"t1\": 4, \"dt\": \"2020-01-04T00:42:00.123\"," + + " \"i1\": null}", + "{\"ts\": 5, \"s1\": \"1005\", \"d1\": 4, \"t1\": 5, \"dt\": \"2020-01-05T00:42:00.123\"," + + " \"i1\": 2005}" }; FakeDatasetService.setUp(); @@ -408,14 +447,19 @@ public void testE2E_withEnforceSamePartitionKeyEnabled_producesRenamedColumns() String[] expectedRecords1 = new String[] { - "{\"ts_pkey\": 1, \"s1\": \"1001\", \"d1\": 0, \"i1\": 2001}", - "{\"ts_pkey\": 2, \"s1\": \"1002\", \"d1\": 1, \"i1\": 2002}" + "{\"ts_pkey\": 1, \"s1\": \"1001\", \"d1\": 0, \"t1\": 1, \"dt\":" + + " \"2020-01-01T00:42:00.123\", \"i1\": 2001}", + "{\"ts_pkey\": 2, \"s1\": \"1002\", \"d1\": 1, \"t1\": 2, \"dt\":" + + " \"2020-01-02T00:42:00.123\", \"i1\": 2002}" }; String[] expectedRecords2 = new String[] { - "{\"ts_pkey\": 3, \"s1\": \"1003\", \"d1\": 2, \"i1\": 2003}", - "{\"ts_pkey\": 4, \"s1\": \"1004\", \"d1\": 3, \"i1\": null}", - "{\"ts_pkey\": 5, \"s1\": \"1005\", \"d1\": 4, \"i1\": 2005}" + "{\"ts_pkey\": 3, \"s1\": \"1003\", \"d1\": 2, \"t1\": 3, \"dt\":" + + " \"2020-01-03T00:42:00.123\", \"i1\": 2003}", + "{\"ts_pkey\": 4, \"s1\": \"1004\", \"d1\": 3, \"t1\": 4, \"dt\":" + + " \"2020-01-04T00:42:00.123\", \"i1\": null}", + "{\"ts_pkey\": 5, \"s1\": \"1005\", \"d1\": 4, \"t1\": 5, \"dt\":" + + " \"2020-01-05T00:42:00.123\", \"i1\": 2005}" }; PAssert.that(actualRecords1).containsInAnyOrder(expectedRecords1); From 6d8a576810a05d2b47767885a4bd266a01f315c5 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Thu, 7 Apr 2022 15:34:53 -0700 Subject: [PATCH 098/145] Add template for writing Spanner Change Stream records into GCS. PiperOrigin-RevId: 440217254 --- v2/googlecloud-to-googlecloud/README.md | 7 + .../docs/SpannerChangeStreamsToGcs/README.md | 221 ++++++++++++++ v2/googlecloud-to-googlecloud/pom.xml | 14 + .../SpannerChangeStreamsToGcsOptions.java | 121 ++++++++ .../templates/SpannerChangeStreamsToGcs.java | 131 ++++++++ ...FileFormatFactorySpannerChangeStreams.java | 107 +++++++ .../WriteDataChangeRecordsToGcsAvro.java | 2 +- .../WriteDataChangeRecordsToGcsText.java | 2 +- ...ner-changestreams-to-gcs-command-spec.json | 7 + .../teleport/v2/spanner/IntegrationTest.java | 19 ++ .../v2/spanner/SpannerServerResource.java | 92 ++++++ .../SpannerChangeStreamsToGcsTest.java | 281 ++++++++++++++++++ 12 files changed, 1002 insertions(+), 2 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/docs/SpannerChangeStreamsToGcs/README.md create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-gcs-command-spec.json create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/IntegrationTest.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/SpannerServerResource.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java diff --git a/v2/googlecloud-to-googlecloud/README.md b/v2/googlecloud-to-googlecloud/README.md index 29107a9140..baaac33b42 100644 --- a/v2/googlecloud-to-googlecloud/README.md +++ b/v2/googlecloud-to-googlecloud/README.md @@ -1,3 +1,10 @@ # Google Cloud to Google Cloud Dataflow Templates TODO: Update docs. + +A collection of Dataflow Flex Templates to stream data (Avro, Proto, +Json etc.) from Google Cloud to Google Cloud. + +* [Spanner change streams to GCS](docs/SpannerChangeStreamsToGcs/README.md) + +Please refer to the links above for more details on the specific template. diff --git a/v2/googlecloud-to-googlecloud/docs/SpannerChangeStreamsToGcs/README.md b/v2/googlecloud-to-googlecloud/docs/SpannerChangeStreamsToGcs/README.md new file mode 100644 index 0000000000..59f07af35b --- /dev/null +++ b/v2/googlecloud-to-googlecloud/docs/SpannerChangeStreamsToGcs/README.md @@ -0,0 +1,221 @@ +# Spanner Change Streams To Google Cloud Storage Dataflow Template + +The [SpannerChangeStreamsToGCS] +(src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGCS.java) +pipeline reads messages from Cloud Spanner Change Streams and stores them in a +Google Cloud Storage bucket using the specified file format. + +Data will be bucketed by timestamp into different windows. By default, the +window size is 5 minutes. + +The data can be stored in a Text or Avro File Format. + +NOTE: This template is currently unreleased. If you wish to use it now, you +will need to follow the steps outlined below to add it to and run it from +your own Google Cloud project. Make sure to be in the /v2 directory. + +## Getting started + +### Requirements +* Java 8 +* Maven +* Spanner Instance exists +* Spanner Database exists +* Spanner Metadata Instance exists +* Spanner Metadata Database exists +* Spanner change stream exists +* Google Cloud Storage output bucket exists. + +### Building Template +This is a Flex Template meaning that the pipeline code will be containerized and the container will be +run on Dataflow. + +##### Building Container Image + +* Set environment variables that will be used in the build process. + +```sh +export PROJECT=project +export IMAGE_NAME=googlecloud-to-googlecloud +export BUCKET_NAME=gs://bucket +export TARGET_GCR_IMAGE=gcr.io/${PROJECT}/images/${IMAGE_NAME} +export BASE_CONTAINER_IMAGE=gcr.io/dataflow-templates-base/java8-template-launcher-base +export BASE_CONTAINER_IMAGE_VERSION=latest +export TEMPLATE_MODULE=googlecloud-to-googlecloud +export COMMAND_MODULE=spanner-changestreams-to-gcs +export APP_ROOT=/template/${COMMAND_MODULE} +export COMMAND_SPEC=${APP_ROOT}/resources/${COMMAND_MODULE}-command-spec.json +export TEMPLATE_IMAGE_SPEC=${BUCKET_NAME}/images/${COMMAND_MODULE}-image-spec.json +``` +* Build and push image to Google Container Repository from the v2 directory + +```sh +mvn clean package -Dimage=${TARGET_GCR_IMAGE} \ + -Dbase-container-image=${BASE_CONTAINER_IMAGE} \ + -Dbase-container-image.version=${BASE_CONTAINER_IMAGE_VERSION} \ + -Dapp-root=${APP_ROOT} \ + -Dcommand-spec=${COMMAND_SPEC} \ + -am -pl ${TEMPLATE_MODULE} +``` + +#### Creating Image Spec + +Create file in Cloud Storage with path to container image in Google Container Repository. + +```sh +echo '{ + "image": "'${TARGET_GCR_IMAGE}'", + "metadata":{"name":"Spanner change streams to GCS", + "description":"Streaming pipeline. Streams change stream records and writes them into a Google Cloud Storage bucket. Note the created pipeline will run on Dataflow Runner V2", + "parameters":[ + { + "label": "Spanner Project ID", + "help_text": "Project to read change streams from. The default for this parameter is the project where the Dataflow pipeline is running.", + "name": "spannerProjectId", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Spanner instance ID", + "help_text": "The Spanner instance to read change streams from.", + "name": "spannerInstanceId", + "param_type": "TEXT" + }, + { + "label": "Spanner database ID", + "help_text": "The Spanner database to read change streams from.", + "name": "spannerDatabaseId", + "param_type": "TEXT" + }, + { + "label": "Spanner metadata instance ID", + "help_text": "The Spanner instance to use for the change stream metadata table.", + "name": "spannerMetadataInstanceId", + "param_type": "TEXT" + }, + { + "label": "Spanner metadata database ID", + "help_text": "The Spanner database to use for the change stream metadata table.", + "name": "spannerMetadataDatabaseId", + "param_type": "TEXT" + }, + { + "label": "Spanner change stream", + "help_text": "The Spanner change stream to read from.", + "name": "spannerChangeStreamName", + "param_type": "TEXT" + }, + { + "label": "Pipeline start time", + "help_text": "The starting DateTime to use for reading change streams (https://tools.ietf.org/html/rfc3339). Defaults to now.", + "name": "startTimestamp", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Pipeline end time", + "help_text": "The ending DateTime to use for reading change streams (https://tools.ietf.org/html/rfc3339). Defaults to max, which represents an infinite time in the future.", + "name": "endTimestamp", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Output file format", + "help_text": "The format of the output GCS file. Allowed formats are TEXT, AVRO. Default is AVRO.", + "name": "outputFileFormat", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "Window duration", + "help_text": "The window duration in which data will be written. Defaults to 5m. Allowed formats are: s (for seconds, example: 5s), m (for minutes, example: 12m), h (for hours, example: 2h).", + "name": "windowDuration", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "The RPC priority.", + "help_text": "Priority for Spanner RPC invocations. Defaults to HIGH. Allowed priorities are LOW, MEDIUM,HIGH. Defaults to HIGH", + "name": "rpcPriority", + "is_optional": true, + "param_type": "TEXT" + }, + { + "label": "File location for change stream output in Cloud Storage", + "help_text": "This is the file location for change stream output in Cloud Storage, in the format: gs://${BUCKET}/${ROOT_PATH}/.", + "name": "outputDirectory", + "param_type": "TEXT" + }, + { + "label": "The filename prefix of the files to write to", + "help_text": "The filename prefix of the files to write to. Default file prefix is set to \"output\"", + "name": "outputFilenamePrefix", + "param_type": "TEXT", + "is_optional": true + }, + { + "label": "Maximum output shards", + "help_text": "The maximum number of output shards produced when writing. Default number is runner defined", + "name": "numShards", + "param_type": "TEXT", + "is_optional": true + } + ]}, + "sdk_info": { + "language": "JAVA" + } +}'> image-spec.json +gsutil cp image-spec.json ${TEMPLATE_IMAGE_SPEC} +``` + + +### Testing Template + +The template unit tests can be run using: +```sh +mvn test +``` + +### Executing Template + +The template requires the following parameters: +* spannerInstanceId: The Spanner Instance ID +* spannerDatabaseId: The Spanner database ID +* spannerMetadataInstanceId: The Spanner Metadata Instance ID. +* spannerMetadataDatabaseId: The Spanner Metadata Database ID. +* spannerChangeStream: The Spanner change stream. +* outputDirectory: The GCS Output Directory. + +The template has the following optional parameters: +* startTimestamp: The starting DateTime to use for reading change streams. Defaults to now. +* endTimestamp: The ending DateTime to use for reading change streams. Defaults to infinite. +* outputFileFormat: The format of the output GCS file. Defaults to AVRO. Can be either TEXT or AVRO. +* windowDuration: The window duration in which data will be written. Defaults to 5m. +* rpcPriority: The priority for Spanner RPC priority. Defaults to HIGH. +* outputFileNamePrefix: The output filename prefix. Defaults to "output" +* numShards: the maximum number of output shards produced when writing. Default is 1. + +Template can be executed using the following gcloud command: + +```sh +export JOB_NAME="${TEMPLATE_MODULE}-`date +%Y%m%d-%H%M%S-%N`" + +export SPANNER_INSTANCE=spanner-instance +export SPANNER_DATABASE=spanner-database +export SPANNER_METADATA_INSTANCE=spanner-metadata-instance +export SPANNER_METADATA_DATABASE=spanner-metadata-database +export SPANNER_CHANGE_STREAM=spanner-changestream +export OUTPUT_DIRECTORY=${BUCKET_NAME}/output-directory/ +export OUTPUT_FILE_FORMAT=TEXT +gcloud beta dataflow flex-template run ${JOB_NAME} \ + --project=${PROJECT} --region=us-central1 \ + --template-file-gcs-location=${TEMPLATE_IMAGE_SPEC} \ + --parameters ^~^spannerInstanceId=${SPANNER_INSTANCE}~spannerDatabaseId=${SPANNER_DATABASE}~spannerMetadataInstanceId=${SPANNER_METADATA_INSTANCE}~spannerMetadataDatabaseId=${SPANNER_METADATA_DATABASE}~spannerChangeStreamName=${SPANNER_CHANGE_STREAM}~outputDirectory=${OUTPUT_DIRECTORY}~outputFileFormat=${OUTPUT_FILE_FORMAT} + +``` + +OPTIONAL Dataflow Params: + +--numWorkers=2 +--maxNumWorkers=10 +--workerMachineType=n1-highcpu-4 diff --git a/v2/googlecloud-to-googlecloud/pom.xml b/v2/googlecloud-to-googlecloud/pom.xml index a0b0a0a56d..add2736f93 100644 --- a/v2/googlecloud-to-googlecloud/pom.xml +++ b/v2/googlecloud-to-googlecloud/pom.xml @@ -40,6 +40,20 @@ ${mvn-target-dir} + + + com.google.cloud.tools + jib-maven-plugin + + + package + + build + + + + + diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java new file mode 100644 index 0000000000..c9408968e7 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java @@ -0,0 +1,121 @@ +/* + * 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.v2.options; + +import com.google.cloud.spanner.Options.RpcPriority; +import com.google.cloud.teleport.v2.transforms.WriteDataChangeRecordsToGcsAvro; +import com.google.cloud.teleport.v2.transforms.WriteDataChangeRecordsToGcsText; +import com.google.cloud.teleport.v2.utils.WriteToGCSUtility.FileFormat; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link SpannerChangeStreamsToGcsOptions} interface provides the custom execution options + * passed by the executor at the command-line. + */ +public interface SpannerChangeStreamsToGcsOptions + extends DataflowPipelineOptions, + WriteDataChangeRecordsToGcsAvro.WriteToGcsAvroOptions, + WriteDataChangeRecordsToGcsText.WriteToGcsTextOptions { + + @Description( + "Project to read change streams from. The default for this parameter is the project where the" + + " Dataflow pipeline is running.") + @Default.String("") + String getSpannerProjectId(); + + void setSpannerProjectId(String projectId); + + @Description("The Spanner instance to read from.") + @Validation.Required + String getSpannerInstanceId(); + + void setSpannerInstanceId(String spannerInstanceId); + + @Description("The Spanner database to read from.") + @Validation.Required + String getSpannerDatabaseId(); + + void setSpannerDatabaseId(String spannerDatabaseId); + + @Description("The Spanner instance to use for the change stream metadata table.") + @Validation.Required + String getSpannerMetadataInstanceId(); + + void setSpannerMetadataInstanceId(String spannerMetadataInstanceId); + + @Description("The Spanner database to use for the change stream metadata table.") + @Validation.Required + String getSpannerMetadataDatabaseId(); + + void setSpannerMetadataDatabaseId(String spannerMetadataDatabaseId); + + @Description("The Spanner change stream to read from.") + @Validation.Required + String getSpannerChangeStreamName(); + + void setSpannerChangeStreamName(String spannerChangeStreamName); + + @Description( + "The starting DateTime to use for reading change streams" + + " (https://tools.ietf.org/html/rfc3339). Defaults to now.") + @Default.String("") + String getStartTimestamp(); + + void setStartTimestamp(String startTimestamp); + + @Description( + "The ending DateTime to use for reading change streams" + + " (https://tools.ietf.org/html/rfc3339). The default value is \"max\", which represents" + + " an infinite time in the future.") + @Default.String("") + String getEndTimestamp(); + + void setEndTimestamp(String startTimestamp); + + @Description("Spanner host endpoint (only used for testing).") + @Default.String("https://spanner.googleapis.com") + String getSpannerHost(); + + void setSpannerHost(String value); + + @Description("The format of the output GCS file. Allowed formats are TEXT, AVRO. Default is AVRO") + @Default.Enum("AVRO") + FileFormat getOutputFileFormat(); + + void setOutputFileFormat(FileFormat outputFileFormat); + + @Description( + "The window duration in which data will be written. Defaults to 5m. " + + "Allowed formats are: " + + "s (for seconds, example: 5s), " + + "m (for minutes, example: 12m), " + + "h (for hours, example: 2h).") + @Default.String("5m") + String getWindowDuration(); + + void setWindowDuration(String windowDuration); + + @Description( + "Priority for Spanner RPC invocations. Defaults to HIGH. Allowed priorites are LOW, MEDIUM," + + " HIGH.") + @Default.Enum("HIGH") + RpcPriority getRpcPriority(); + + void setRpcPriority(RpcPriority rpcPriority); +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java new file mode 100644 index 0000000000..0653b047f8 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java @@ -0,0 +1,131 @@ +/* + * Copyright (C) 2022 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.v2.templates; + +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Options.RpcPriority; +import com.google.cloud.teleport.v2.options.SpannerChangeStreamsToGcsOptions; +import com.google.cloud.teleport.v2.transforms.FileFormatFactorySpannerChangeStreams; +import com.google.cloud.teleport.v2.utils.DurationUtils; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; +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.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link SpannerChangeStreamsToGcs} pipeline streams change stream record(s) and stores to + * Google Cloud Storage bucket in user specified format. The sink data can be stored in a Text or + * Avro file format. + */ +public class SpannerChangeStreamsToGcs { + private static final Logger LOG = LoggerFactory.getLogger(SpannerChangeStreamsToGcs.class); + private static final String USE_RUNNER_V2_EXPERIMENT = "use_runner_v2"; + + public static void main(String[] args) { + LOG.info("Starting Input Files to GCS"); + + SpannerChangeStreamsToGcsOptions options = + PipelineOptionsFactory.fromArgs(args).as(SpannerChangeStreamsToGcsOptions.class); + + run(options); + } + + private static String getProjectId(SpannerChangeStreamsToGcsOptions options) { + return options.getSpannerProjectId().isEmpty() + ? options.getProject() + : options.getSpannerProjectId(); + } + + public static PipelineResult run(SpannerChangeStreamsToGcsOptions options) { + LOG.info("Requested File Format is " + options.getOutputFileFormat()); + options.setStreaming(true); + options.setEnableStreamingEngine(true); + options.setAutoscalingAlgorithm( + DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE); + + final Pipeline pipeline = Pipeline.create(options); + + // Get the Spanner project, instance, database, and change stream parameters. + String projectId = getProjectId(options); + String instanceId = options.getSpannerInstanceId(); + String databaseId = options.getSpannerDatabaseId(); + String metadataInstanceId = options.getSpannerMetadataInstanceId(); + String metadataDatabaseId = options.getSpannerMetadataDatabaseId(); + String changeStreamName = options.getSpannerChangeStreamName(); + + // Retrieve and parse the start / end timestamps. + Timestamp startTimestamp = + options.getStartTimestamp().isEmpty() + ? Timestamp.now() + : Timestamp.parseTimestamp(options.getStartTimestamp()); + Timestamp endTimestamp = + options.getEndTimestamp().isEmpty() + ? Timestamp.MAX_VALUE + : Timestamp.parseTimestamp(options.getEndTimestamp()); + + // Add use_runner_v2 to the experiments option, since Change Streams connector is only supported + // on Dataflow runner v2. + List experiments = options.getExperiments(); + if (experiments == null) { + experiments = new ArrayList<>(); + } + boolean hasUseRunnerV2 = false; + for (String experiment : experiments) { + if (experiment.toLowerCase().equals(USE_RUNNER_V2_EXPERIMENT)) { + hasUseRunnerV2 = true; + break; + } + } + if (!hasUseRunnerV2) { + experiments.add(USE_RUNNER_V2_EXPERIMENT); + } + options.setExperiments(experiments); + + final RpcPriority rpcPriority = options.getRpcPriority(); + pipeline + .apply( + SpannerIO.readChangeStream() + .withSpannerConfig( + SpannerConfig.create() + .withHost(ValueProvider.StaticValueProvider.of(options.getSpannerHost())) + .withProjectId(projectId) + .withInstanceId(instanceId) + .withDatabaseId(databaseId)) + .withMetadataInstance(metadataInstanceId) + .withMetadataDatabase(metadataDatabaseId) + .withChangeStreamName(changeStreamName) + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp) + .withRpcPriority(rpcPriority)) + .apply( + "Creating " + options.getWindowDuration() + " Window", + Window.into(FixedWindows.of(DurationUtils.parseDuration(options.getWindowDuration())))) + .apply( + "Write To GCS", + FileFormatFactorySpannerChangeStreams.newBuilder().setOptions(options).build()); + + return pipeline.run(); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java new file mode 100644 index 0000000000..c669b8d57a --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java @@ -0,0 +1,107 @@ +/* + * 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.v2.transforms; + +import com.google.auto.value.AutoValue; +import com.google.cloud.teleport.v2.options.SpannerChangeStreamsToGcsOptions; +import com.google.cloud.teleport.v2.utils.WriteToGCSUtility.FileFormat; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.POutput; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link FileFormatFactorySpannerChangeStreams} class is a {@link PTransform} that takes in + * {@link PCollection} of DataChangeRecords. The transform writes these records to GCS file(s) in + * user specified format. + */ +@AutoValue +public abstract class FileFormatFactorySpannerChangeStreams + extends PTransform, POutput> { + + /** Logger for class. */ + private static final Logger LOG = + LoggerFactory.getLogger(FileFormatFactorySpannerChangeStreams.class); + + public static WriteToGcsBuilder newBuilder() { + return new AutoValue_FileFormatFactorySpannerChangeStreams.Builder(); + } + + public abstract SpannerChangeStreamsToGcsOptions options(); + + public abstract FileFormat outputFileFormat(); + + @Override + public POutput expand(PCollection records) { + POutput output = null; + + final String errorMessage = + "Invalid output format:" + outputFileFormat() + ". Supported output formats: TEXT, AVRO"; + + /* + * Calls appropriate class Builder to performs PTransform based on user provided File Format. + */ + switch (outputFileFormat()) { + case AVRO: + output = + records.apply( + "Write Avro File(s)", + WriteDataChangeRecordsToGcsAvro.newBuilder() + .withOutputDirectory(options().getOutputDirectory()) + .withOutputFilenamePrefix(options().getOutputFilenamePrefix()) + .setNumShards(options().getNumShards()) + .withTempLocation(options().getTempLocation()) + .build()); + break; + case TEXT: + output = + records.apply( + "Write Text File(s)", + WriteDataChangeRecordsToGcsText.newBuilder() + .withOutputDirectory(options().getOutputDirectory()) + .withOutputFilenamePrefix(options().getOutputFilenamePrefix()) + .setNumShards(options().getNumShards()) + .withTempLocation(options().getTempLocation()) + .build()); + break; + + default: + LOG.info(errorMessage); + throw new IllegalArgumentException(errorMessage); + } + return output; + } + + /** Builder for {@link FileFormatFactory}. */ + @AutoValue.Builder + public abstract static class WriteToGcsBuilder { + + public abstract WriteToGcsBuilder setOptions(SpannerChangeStreamsToGcsOptions options); + + public abstract WriteToGcsBuilder setOutputFileFormat(FileFormat outputFileFormat); + + abstract SpannerChangeStreamsToGcsOptions options(); + + abstract FileFormatFactorySpannerChangeStreams autoBuild(); + + public FileFormatFactorySpannerChangeStreams build() { + setOutputFileFormat(options().getOutputFileFormat()); + return autoBuild(); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java index 570d7ed9e0..8ea07c1d06 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java @@ -103,7 +103,7 @@ public interface WriteToGcsAvroOptions extends PipelineOptions { @Description( "The maximum number of output shards produced when writing. Default number is runner" + " defined.") - @Default.Integer(1) + @Default.Integer(20) Integer getNumShards(); void setNumShards(Integer numShards); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java index 79b1e89c35..b9320933fa 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java @@ -121,7 +121,7 @@ public interface WriteToGcsTextOptions extends PipelineOptions { @Description( "The maximum number of output shards produced when writing. Default number is runner" + " defined.") - @Default.Integer(1) + @Default.Integer(20) Integer getNumShards(); void setNumShards(Integer numShards); diff --git a/v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-gcs-command-spec.json b/v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-gcs-command-spec.json new file mode 100644 index 0000000000..0b9fe329ce --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-gcs-command-spec.json @@ -0,0 +1,7 @@ +{ + "mainClass": "com.google.cloud.teleport.v2.templates.SpannerChangeStreamsToGcs", + "classPath": "/template/spanner-changestreams-to-gcs/*:/template/spanner-changestreams-to-gcs/libs/*:/template/spanner-changestreams-to-gcs/classes", + "defaultParameterValues": { + "labels": "{\"goog-dataflow-provided-template-type\":\"flex\", \"goog-dataflow-provided-template-name\":\"spanner-changestreams-to-gcs\"}" + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/IntegrationTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/IntegrationTest.java new file mode 100644 index 0000000000..5690e87f9f --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/IntegrationTest.java @@ -0,0 +1,19 @@ +/* + * Copyright (C) 2022 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.v2.spanner; + +/** Mark the Spanner integration tests. */ +public interface IntegrationTest {} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/SpannerServerResource.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/SpannerServerResource.java new file mode 100644 index 0000000000..1229a2ead6 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/spanner/SpannerServerResource.java @@ -0,0 +1,92 @@ +/* + * Copyright (C) 2020 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.v2.spanner; + +import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.ErrorCode; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerException; +import com.google.cloud.spanner.SpannerOptions; +import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.options.ValueProvider; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Facilitates setup and deletion of a Spanner database for integration tests. */ +public class SpannerServerResource extends ExternalResource { + private final String projectId = "span-cloud-testing"; + private final String instanceId = "changestream"; + private final String host = "https://spanner.googleapis.com"; + private static final int TIMEOUT_MINUTES = 10; + private static final Logger LOG = LoggerFactory.getLogger(SpannerServerResource.class); + + private Spanner client; + private DatabaseAdminClient databaseAdminClient; + + @Override + protected void before() throws Exception { + SpannerOptions spannerOptions = + SpannerOptions.newBuilder().setProjectId(projectId).setHost(host).build(); + client = spannerOptions.getService(); + databaseAdminClient = client.getDatabaseAdminClient(); + } + + @Override + protected void after() { + client.close(); + } + + public void createDatabase(String dbName, Iterable ddlStatements) throws Exception { + // Waits for create database to complete. + databaseAdminClient + .createDatabase(instanceId, dbName, ddlStatements) + .get(TIMEOUT_MINUTES, TimeUnit.MINUTES); + ; + } + + public void updateDatabase(String dbName, Iterable ddlStatements) throws Exception { + databaseAdminClient + .updateDatabaseDdl(instanceId, dbName, ddlStatements, /*operationId=*/ null) + .get(TIMEOUT_MINUTES, TimeUnit.MINUTES); + } + + public void dropDatabase(String dbName) { + try { + databaseAdminClient.dropDatabase(instanceId, dbName); + } catch (SpannerException e) { + // Does not exist, ignore. + if (!e.getErrorCode().equals(ErrorCode.NOT_FOUND)) { + LOG.warn("Failed to drop database " + dbName); + } + } + } + + public DatabaseClient getDbClient(String dbName) { + return client.getDatabaseClient(DatabaseId.of(projectId, instanceId, dbName)); + } + + public SpannerConfig getSpannerConfig(String dbName) { + return SpannerConfig.create() + .withProjectId(projectId) + .withInstanceId(instanceId) + .withDatabaseId(dbName) + .withHost(ValueProvider.StaticValueProvider.of(host)); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java new file mode 100644 index 0000000000..8c7815415c --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java @@ -0,0 +1,281 @@ +/* + * Copyright (C) 2022 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.v2.templates; + +import static com.google.cloud.teleport.v2.templates.SpannerChangeStreamsToGcs.run; +import static org.junit.Assert.assertEquals; + +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Options.RpcPriority; +import com.google.cloud.teleport.v2.options.SpannerChangeStreamsToGcsOptions; +import com.google.cloud.teleport.v2.spanner.IntegrationTest; +import com.google.cloud.teleport.v2.spanner.SpannerServerResource; +import com.google.cloud.teleport.v2.transforms.FileFormatFactorySpannerChangeStreams; +import com.google.cloud.teleport.v2.utils.DurationUtils; +import com.google.cloud.teleport.v2.utils.WriteToGCSUtility.FileFormat; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test class for {@link SpannerChangeStreamsToGcsTest}. */ +@RunWith(JUnit4.class) +public final class SpannerChangeStreamsToGcsTest { + + /** Rule for exception testing. */ + @Rule public ExpectedException exception = ExpectedException.none(); + + @Rule public final TemporaryFolder tmpDir = new TemporaryFolder(); + + /** Rule for Spanner server resource. */ + @Rule public final SpannerServerResource spannerServer = new SpannerServerResource(); + + /** Rule for pipeline testing. */ + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static final String AVRO_FILENAME_PREFIX = "avro-output-"; + private static final Integer NUM_SHARDS = 1; + private static final String TEST_PROJECT = "span-cloud-testing"; + private static final String TEST_INSTANCE = "changestream"; + private static final String TEST_DATABASE_PREFIX = "testdbchangestreams"; + private static final String TEST_TABLE = "Users"; + private static final String TEST_CHANGE_STREAM = "UsersStream"; + private static final int MAX_TABLE_NAME_LENGTH = 29; + + private static String fakeDir; + private static String fakeTempLocation; + private static final String FILENAME_PREFIX = "filenamePrefix"; + + @Before + public void setup() throws Exception { + fakeDir = tmpDir.newFolder("output").getAbsolutePath(); + fakeTempLocation = tmpDir.newFolder("temporaryLocation").getAbsolutePath(); + } + + @SuppressWarnings("DefaultAnnotationParam") + private static class VerifyDataChangeRecordAvro + implements SerializableFunction, Void> { + @Override + public Void apply(Iterable actualIter) { + // Make sure actual is the right length, and is a + // subset of expected. + List actual = new ArrayList<>(); + for (DataChangeRecord s : actualIter) { + actual.add(s); + assertEquals(TEST_TABLE, s.getTableName()); + } + assertEquals(actual.size(), 1); + return null; + } + } + + private String generateDatabaseName() { + return TEST_DATABASE_PREFIX + + "_" + + RandomStringUtils.randomNumeric( + MAX_TABLE_NAME_LENGTH - 1 - TEST_DATABASE_PREFIX.length()); + } + + /** + * Test whether {@link FileFormatFactory} maps the output file format to the transform to be + * carried out. And throws illegal argument exception if invalid file format is passed. + */ + @Test + public void testFileFormatFactoryInvalid() { + + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid output format:PARQUET. Supported output formats: TEXT, AVRO"); + + SpannerChangeStreamsToGcsOptions options = + PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); + options.setOutputFileFormat(FileFormat.PARQUET); + options.setOutputDirectory(fakeDir); + options.setOutputFilenamePrefix(FILENAME_PREFIX); + options.setNumShards(NUM_SHARDS); + options.setTempLocation(fakeTempLocation); + + Pipeline p = Pipeline.create(options); + + Timestamp startTimestamp = Timestamp.now(); + Timestamp endTimestamp = Timestamp.now(); + + p + // Reads from the change stream + .apply( + SpannerIO.readChangeStream() + .withSpannerConfig( + SpannerConfig.create() + .withProjectId("project") + .withInstanceId("instance") + .withDatabaseId("db")) + .withMetadataInstance("instance") + .withMetadataDatabase("db") + .withChangeStreamName("changestream") + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp) + .withRpcPriority(RpcPriority.HIGH)) + .apply( + "Creating " + options.getWindowDuration() + " Window", + Window.into(FixedWindows.of(DurationUtils.parseDuration(options.getWindowDuration())))) + .apply( + "Write To GCS", + FileFormatFactorySpannerChangeStreams.newBuilder().setOptions(options).build()); + + p.run(); + } + + @Test + public void testInvalidWindowDuration() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("The window duration must be greater than 0!"); + SpannerChangeStreamsToGcsOptions options = + PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); + options.setOutputFileFormat(FileFormat.AVRO); + options.setOutputDirectory(fakeDir); + options.setOutputFilenamePrefix(FILENAME_PREFIX); + options.setNumShards(NUM_SHARDS); + options.setTempLocation(fakeTempLocation); + options.setWindowDuration("invalidWindowDuration"); + + Pipeline p = Pipeline.create(options); + + Timestamp startTimestamp = Timestamp.now(); + Timestamp endTimestamp = Timestamp.now(); + + p + // Reads from the change stream + .apply( + SpannerIO.readChangeStream() + .withSpannerConfig( + SpannerConfig.create() + .withProjectId("project") + .withInstanceId("instance") + .withDatabaseId("db")) + .withMetadataInstance("instance") + .withMetadataDatabase("db") + .withChangeStreamName("changestream") + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp) + .withRpcPriority(RpcPriority.HIGH)) + .apply( + "Creating " + options.getWindowDuration() + " Window", + Window.into(FixedWindows.of(DurationUtils.parseDuration(options.getWindowDuration())))) + .apply( + "Write To GCS", + FileFormatFactorySpannerChangeStreams.newBuilder().setOptions(options).build()); + + p.run(); + } + + @Test + @Category(IntegrationTest.class) + // This test can only be run locally with the following command: + // mvn -Dexcluded.spanner.tests="" -Dtest=SpannerChangeStreamsToGcsTest test + // TODO(nancyxu): Add an integration test for writing into GCS text when the connector can be + // run in parallel testing. Should happen after this PR is submitted: + // https://github.com/apache/beam/pull/17036 + public void testWriteToGCSAvro() throws Exception { + // Create a test database. + String testDatabase = generateDatabaseName(); + fakeDir = tmpDir.newFolder("output").getAbsolutePath(); + fakeTempLocation = tmpDir.newFolder("temporaryLocation").getAbsolutePath(); + + spannerServer.dropDatabase(testDatabase); + + // Create a table. + List statements = new ArrayList(); + final String createTable = + "CREATE TABLE " + + TEST_TABLE + + " (" + + "user_id INT64 NOT NULL," + + "name STRING(MAX) " + + ") PRIMARY KEY(user_id)"; + final String createChangeStream = "CREATE CHANGE STREAM " + TEST_CHANGE_STREAM + " FOR Users"; + statements.add(createTable); + statements.add(createChangeStream); + spannerServer.createDatabase(testDatabase, statements); + + Timestamp startTimestamp = Timestamp.now(); + + // Create a mutation for the table that will generate 1 data change record. + List mutations = new ArrayList<>(); + mutations.add( + Mutation.newInsertBuilder(TEST_TABLE).set("user_id").to(1).set("name").to("Name1").build()); + mutations.add( + Mutation.newInsertBuilder(TEST_TABLE).set("user_id").to(2).set("name").to("Name2").build()); + + spannerServer.getDbClient(testDatabase).write(mutations); + + Timestamp endTimestamp = Timestamp.now(); + + SpannerChangeStreamsToGcsOptions options = + PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); + options.setSpannerProjectId(TEST_PROJECT); + options.setSpannerInstanceId(TEST_INSTANCE); + options.setSpannerDatabaseId(testDatabase); + options.setSpannerMetadataInstanceId(TEST_INSTANCE); + options.setSpannerMetadataDatabaseId(testDatabase); + options.setSpannerChangeStreamName(TEST_CHANGE_STREAM); + + options.setStartTimestamp(startTimestamp.toString()); + options.setEndTimestamp(endTimestamp.toString()); + List experiments = new ArrayList(); + options.setExperiments(experiments); + + options.setOutputFileFormat(FileFormat.AVRO); + options.setOutputDirectory(fakeDir); + options.setOutputFilenamePrefix(AVRO_FILENAME_PREFIX); + options.setNumShards(NUM_SHARDS); + options.setTempLocation(fakeTempLocation); + + // Run the pipeline. + PipelineResult result = run(options); + result.waitUntilFinish(); + + // Read from the output Avro file to assert that 1 data change record has been generated. + PCollection dataChangeRecords = + pipeline.apply( + "readRecords", + AvroIO.read(DataChangeRecord.class).from(fakeDir + "/avro-output-*.avro")); + PAssert.that(dataChangeRecords).satisfies(new VerifyDataChangeRecordAvro()); + pipeline.run(); + + // Drop the database. + spannerServer.dropDatabase(testDatabase); + } +} From 59a37e1697d38a08b094fe98da77a2e91cfa956e Mon Sep 17 00:00:00 2001 From: dhercher Date: Fri, 8 Apr 2022 09:18:40 -0700 Subject: [PATCH 099/145] DatastreamToBigQuery uses a stateful transform to handle Oracle rollbacks. PiperOrigin-RevId: 440381814 --- .../teleport/v2/values/DatastreamRow.java | 9 +- .../v2/templates/DataStreamToBigQuery.java | 26 ++- .../v2/transforms/StatefulRowCleaner.java | 208 ++++++++++++++++++ .../teleport/v2/transforms/package-info.java | 18 ++ 4 files changed, 256 insertions(+), 5 deletions(-) create mode 100644 v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java create mode 100644 v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java index 783fc2f160..59058c3eec 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java @@ -36,12 +36,13 @@ // @DefaultCoder(FailsafeElementCoder.class) public class DatastreamRow { + public static final String DEFAULT_ORACLE_PRIMARY_KEY = "_metadata_row_id"; + public static final String ORACLE_TRANSACTION_ID_KEY = "_metadata_tx_id"; + private static final Logger LOG = LoggerFactory.getLogger(DatastreamRow.class); private TableRow tableRow; private JsonNode jsonRow; - private static final String DEFAULT_ORACLE_PRIMARY_KEY = "_metadata_row_id"; - private DatastreamRow(TableRow tableRow, JsonNode jsonRow) { this.tableRow = tableRow; this.jsonRow = jsonRow; @@ -164,4 +165,8 @@ public String toString() { return this.tableRow.toString(); } } + + public String getOracleRowId() { + return this.getStringValue(DEFAULT_ORACLE_PRIMARY_KEY); + } } diff --git a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java index 8e3f03f69c..e91472a219 100644 --- a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java +++ b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java @@ -15,6 +15,8 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.transforms.StatefulRowCleaner.RowCleanerDeadLetterQueueSanitizer; + import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.bigquery.TableId; import com.google.cloud.teleport.v2.cdc.dlq.DeadLetterQueueManager; @@ -27,6 +29,7 @@ import com.google.cloud.teleport.v2.cdc.sources.DataStreamIO; import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; import com.google.cloud.teleport.v2.transforms.DLQWriteTransform; +import com.google.cloud.teleport.v2.transforms.StatefulRowCleaner; import com.google.cloud.teleport.v2.transforms.UDFTextTransformer.InputUDFOptions; import com.google.cloud.teleport.v2.transforms.UDFTextTransformer.InputUDFToTableRow; import com.google.cloud.teleport.v2.values.FailsafeElement; @@ -300,6 +303,8 @@ public static PipelineResult run(Options options) { options.getRuntimeRetries(), FAILSAFE_ELEMENT_CODER); + StatefulRowCleaner statefulCleaner = StatefulRowCleaner.of(); + /* * Stage 1: Ingest and Normalize Data to FailsafeElement with JSON Strings * a) Read DataStream data from GCS into JSON String FailsafeElements (datastreamJsonRecords) @@ -349,9 +354,14 @@ public void process( PCollectionTuple tableRowRecords = jsonRecords.apply("UDF to TableRow/udf", failsafeTableRowTransformer); - PCollection shuffledTableRows = + PCollectionTuple cleanedRows = tableRowRecords .get(failsafeTableRowTransformer.transformOut) + .apply("UDF to TableRow/Oracle Cleaner", statefulCleaner); + + PCollection shuffledTableRows = + cleanedRows + .get(statefulCleaner.successTag) .apply( "UDF to TableRow/ReShuffle", Reshuffle.viaRandomKey().withNumBuckets(100)); @@ -428,8 +438,17 @@ public void process( PCollection udfDlqJson = PCollectionList.of(tableRowRecords.get(failsafeTableRowTransformer.udfDeadletterOut)) .and(tableRowRecords.get(failsafeTableRowTransformer.transformDeadletterOut)) - .apply("UDF Failures/Flatten", Flatten.pCollections()) - .apply("UDF Failures/Sanitize", MapElements.via(new StringDeadLetterQueueSanitizer())); + .apply("Transform Failures/Flatten", Flatten.pCollections()) + .apply( + "Transform Failures/Sanitize", + MapElements.via(new StringDeadLetterQueueSanitizer())); + + PCollection rowCleanerJson = + cleanedRows + .get(statefulCleaner.failureTag) + .apply( + "Transform Failures/Oracle Cleaner Failures", + MapElements.via(new RowCleanerDeadLetterQueueSanitizer())); PCollection bqWriteDlqJson = writeResult @@ -437,6 +456,7 @@ public void process( .apply("BigQuery Failures", MapElements.via(new BigQueryDeadLetterQueueSanitizer())); PCollectionList.of(udfDlqJson) + .and(rowCleanerJson) .and(bqWriteDlqJson) .apply("Write To DLQ/Flatten", Flatten.pCollections()) .apply( diff --git a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java new file mode 100644 index 0000000000..5556798452 --- /dev/null +++ b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java @@ -0,0 +1,208 @@ +/* + * Copyright (C) 2021 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.v2.transforms; + +import com.google.api.client.json.JsonFactory; +import com.google.api.client.json.jackson2.JacksonFactory; +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.teleport.v2.cdc.dlq.DeadLetterQueueSanitizer; +import com.google.cloud.teleport.v2.values.DatastreamRow; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; +import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.StateId; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.commons.lang3.RandomStringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@code StatefulRowCleaner} class statefully processes and filters data based on the supplied + * primary keys and sort keys in Datastream TableRow data. + */ +public class StatefulRowCleaner extends PTransform, PCollectionTuple> { + + private static final Logger LOG = LoggerFactory.getLogger(StatefulRowCleaner.class); + private static final String WINDOW_DURATION = "1s"; + + public TupleTag successTag = new TupleTag() {}; + public TupleTag failureTag = new TupleTag() {}; + + private StatefulRowCleaner() {} + + public static StatefulRowCleaner of() { + return new StatefulRowCleaner(); + } + + @Override + public PCollectionTuple expand(PCollection input) { + return input + .apply(ParDo.of(new KeyOnDatastreamRowId())) + .apply( + ParDo.of(new StatefulCleanDatastreamRowFn(failureTag)) + .withOutputTags(successTag, TupleTagList.of(failureTag))); + } + + /** + * The {@code StatefulCleanDatastreamRowFn} class statefully processes and filters data based on + * the supplied primary keys and sort keys in DmlInfo. + */ + public static class KeyOnDatastreamRowId extends DoFn> { + + public KeyOnDatastreamRowId() {} + + @ProcessElement + public void processElement(ProcessContext context) { + TableRow tableRow = context.element(); + DatastreamRow row = DatastreamRow.of(tableRow); + if (row.getSourceType().equals("oracle")) { + context.output(KV.of(row.getOracleRowId(), tableRow)); + } else { + String generatedString = + RandomStringUtils.random( + /* length */ 10, /* useLetters */ true, /* useNumbers */ false); + context.output(KV.of(generatedString, tableRow)); + } + } + } + + /** + * The {@code StatefulCleanDatastreamRowFn} class statefully processes and filters data based on + * the supplied primary keys and sort keys in the TableRow. + * + *

The core usecase is to ensure Oracle transaction rollbacks are supplemented with the correct + * primary key values. + */ + public static class StatefulCleanDatastreamRowFn extends DoFn, TableRow> { + + public final TupleTag failureTag; + private static final String PK_STATE_ID = "pk-state-id"; + private final Distribution distribution = + Metrics.distribution(StatefulCleanDatastreamRowFn.class, "replicationDistribution"); + + @StateId(PK_STATE_ID) + private final StateSpec> myStateSpec = + StateSpecs.value(TableRowJsonCoder.of()); + + public StatefulCleanDatastreamRowFn(TupleTag failureTag) { + this.failureTag = failureTag; + } + + @ProcessElement + public void processElement( + ProcessContext context, @StateId(PK_STATE_ID) ValueState myState) { + TableRow tableRow = context.element().getValue(); + DatastreamRow row = DatastreamRow.of(tableRow); + if (!row.getSourceType().equals("oracle")) { + context.output(tableRow); + return; + } + + List primaryKeys = row.getPrimaryKeys(); + // If the there is no PK or it is ROWID, nothing needs the be done. + if (primaryKeys.isEmpty() + || primaryKeys.equals(ImmutableList.of(DatastreamRow.DEFAULT_ORACLE_PRIMARY_KEY))) { + context.output(tableRow); + return; + } + + // When Primary Keys are not ROWID, use stateful logic. + if (hasPrimaryKeyValues(tableRow, primaryKeys)) { + myState.write(getPrimaryKeysTableRow(tableRow, primaryKeys)); + context.output(tableRow); + return; + } + + TableRow previousTableRow = myState.read(); + if (previousTableRow == null) { + LOG.warn( + "Failed stateful clean requires manual attention for ROWID: {}", + tableRow.get(DatastreamRow.DEFAULT_ORACLE_PRIMARY_KEY)); + context.output(failureTag, tableRow); + return; + } + TableRow newTableRow = tableRow.clone(); + for (String primaryKey : primaryKeys) { + newTableRow.put(primaryKey, previousTableRow.get(primaryKey)); + } + + context.output(newTableRow); + } + + private TableRow getPrimaryKeysTableRow(TableRow tableRow, List primaryKeys) { + TableRow pkRow = new TableRow(); + + for (String primaryKey : primaryKeys) { + pkRow.set(primaryKey, tableRow.get(primaryKey)); + } + + return pkRow; + } + + private Boolean hasPrimaryKeyValues(TableRow tableRow, List primaryKeys) { + for (String primaryKey : primaryKeys) { + if (!tableRow.containsKey(primaryKey) || tableRow.get(primaryKey) == null) { + return false; + } + } + + return true; + } + } + + /** + * The RowCleanerDeadLetterQueueSanitizer cleans and prepares failed row cleaner events to be + * stored in a GCS Dead Letter Queue. NOTE: The input to a Sanitizer is flexible but the output + * must be a String unless your override formatMessage() + */ + public static class RowCleanerDeadLetterQueueSanitizer + extends DeadLetterQueueSanitizer { + + private static final JsonFactory JSON_FACTORY = JacksonFactory.getDefaultInstance(); + + @Override + public String getJsonMessage(TableRow row) { + String message; + try { + row.setFactory(JSON_FACTORY); + message = row.toPrettyString(); + } catch (IOException e) { + // Ignore exception and print bad format + message = String.format("\"%s\"", row.toString()); + } + + return message; + } + + @Override + public String getErrorMessageJson(TableRow row) { + return "Failed stateful clean requires manual attention for ROWID"; + } + } +} diff --git a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java new file mode 100644 index 0000000000..16239e1cd0 --- /dev/null +++ b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2019 Google 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 info for DataStream to BigQuery module. */ +package com.google.cloud.teleport.v2.transforms; From 06afa6694ef7e698777966cd5382eddf10e85926 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Fri, 8 Apr 2022 10:31:26 -0700 Subject: [PATCH 100/145] Implement the conversion between change streams Avro schema and DDL in Cloud Spanner Import/Export pipelines. PiperOrigin-RevId: 440398177 --- .../spanner/AvroSchemaToDdlConverter.java | 33 ++++++++++- .../spanner/DdlToAvroSchemaConverter.java | 17 ++++++ .../spanner/AvroSchemaToDdlConverterTest.java | 59 +++++++++++++++++++ .../spanner/DdlToAvroSchemaConverterTest.java | 51 ++++++++++++++++ 4 files changed, 157 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java index 0996e676df..4951f9527b 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java @@ -19,6 +19,7 @@ import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; +import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.Table; @@ -39,10 +40,12 @@ public class AvroSchemaToDdlConverter { public Ddl toDdl(Collection avroSchemas) { Ddl.Builder builder = Ddl.builder(); for (Schema schema : avroSchemas) { - if (schema.getProp("spannerViewQuery") == null) { - builder.addTable(toTable(null, schema)); - } else { + if (schema.getProp("spannerViewQuery") != null) { builder.addView(toView(null, schema)); + } else if (schema.getProp("spannerChangeStreamForClause") != null) { + builder.addChangeStream(toChangeStream(null, schema)); + } else { + builder.addTable(toTable(null, schema)); } } return builder.build(); @@ -61,6 +64,30 @@ public View toView(String viewName, Schema schema) { return builder.build(); } + public ChangeStream toChangeStream(String changeStreamName, Schema schema) { + if (changeStreamName == null) { + changeStreamName = schema.getName(); + } + LOG.debug("Converting to Ddl changeStreamName {}", changeStreamName); + + ChangeStream.Builder builder = + ChangeStream.builder() + .name(changeStreamName) + .forClause(schema.getProp("spannerChangeStreamForClause")); + + ImmutableList.Builder changeStreamOptions = ImmutableList.builder(); + for (int i = 0; ; i++) { + String spannerOption = schema.getProp("spannerOption_" + i); + if (spannerOption == null) { + break; + } + changeStreamOptions.add(spannerOption); + } + builder.options(changeStreamOptions.build()); + + return builder.build(); + } + public Table toTable(String tableName, Schema schema) { if (tableName == null) { tableName = schema.getName(); diff --git a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java index 675cd7a69d..52d171462e 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner; import com.google.cloud.teleport.spanner.common.NumericUtils; +import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.IndexColumn; @@ -111,6 +112,22 @@ public Collection convert(Ddl ddl) { schemas.add(recordBuilder.fields().endRecord()); } + for (ChangeStream changeStream : ddl.changeStreams()) { + SchemaBuilder.RecordBuilder recordBuilder = + SchemaBuilder.record(changeStream.name()).namespace(this.namespace); + recordBuilder.prop("googleFormatVersion", version); + recordBuilder.prop("googleStorage", "CloudSpanner"); + recordBuilder.prop( + "spannerChangeStreamForClause", + changeStream.forClause() == null ? "" : changeStream.forClause()); + if (changeStream.options() != null) { + for (int i = 0; i < changeStream.options().size(); i++) { + recordBuilder.prop("spannerOption_" + i, changeStream.options().get(i)); + } + } + schemas.add(recordBuilder.fields().endRecord()); + } + return schemas; } diff --git a/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java index f423f11f08..07ffb61913 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java @@ -21,6 +21,8 @@ import static org.junit.Assert.assertThat; import com.google.cloud.teleport.spanner.ddl.Ddl; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import org.apache.avro.Schema; import org.junit.Test; @@ -210,4 +212,61 @@ public void columnOptions() { + " OPTIONS (allow_commit_timestamp=TRUE,my_random_opt=\"1\")," + " ) PRIMARY KEY (`id` ASC)")); } + + @Test + public void changeStreams() { + String avroString1 = + "{" + + " \"type\" : \"record\"," + + " \"name\" : \"ChangeStreamAll\"," + + " \"fields\" : []," + + " \"namespace\" : \"spannertest\"," + + " \"googleStorage\" : \"CloudSpanner\"," + + " \"googleFormatVersion\" : \"booleans\"," + + " \"spannerChangeStreamForClause\" : \"FOR ALL\"," + + " \"spannerOption_0\" : \"retention_period=\\\"7d\\\"\"," + + " \"spannerOption_1\" : \"value_capture_type=\\\"OLD_AND_NEW_VALUES\\\"\"" + + "}"; + String avroString2 = + "{" + + " \"type\" : \"record\"," + + " \"name\" : \"ChangeStreamEmpty\"," + + " \"fields\" : []," + + " \"namespace\" : \"spannertest\"," + + " \"googleStorage\" : \"CloudSpanner\"," + + " \"googleFormatVersion\" : \"booleans\"," + + " \"spannerChangeStreamForClause\" : \"\"" + + "}"; + String avroString3 = + "{" + + " \"type\" : \"record\"," + + " \"name\" : \"ChangeStreamTableColumns\"," + + " \"fields\" : []," + + " \"namespace\" : \"spannertest\"," + + " \"googleStorage\" : \"CloudSpanner\"," + + " \"googleFormatVersion\" : \"booleans\"," + + " \"spannerChangeStreamForClause\" : \"FOR `T1`, `T2`(`c1`, `c2`), `T3`()\"," + + " \"spannerOption_0\" : \"retention_period=\\\"24h\\\"\"" + + "}"; + + Collection schemas = new ArrayList<>(); + Schema.Parser parser = new Schema.Parser(); + schemas.add(parser.parse(avroString1)); + schemas.add(parser.parse(avroString2)); + schemas.add(parser.parse(avroString3)); + + AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(); + Ddl ddl = converter.toDdl(schemas); + assertThat(ddl.changeStreams(), hasSize(3)); + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE CHANGE STREAM `ChangeStreamAll`" + + " FOR ALL" + + " OPTIONS (retention_period=\"7d\", value_capture_type=\"OLD_AND_NEW_VALUES\")" + + " CREATE CHANGE STREAM `ChangeStreamEmpty`" + + " CREATE CHANGE STREAM `ChangeStreamTableColumns`" + + " FOR `T1`, `T2`(`c1`, `c2`), `T3`()" + + " OPTIONS (retention_period=\"24h\")")); + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java index b2458698ac..a17add8f46 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java @@ -28,6 +28,7 @@ import com.google.cloud.teleport.spanner.ddl.View; import com.google.common.collect.ImmutableList; import java.util.Collection; +import java.util.Iterator; import java.util.List; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -405,6 +406,56 @@ public void timestampLogicalTypeTest() { assertThat(fields.get(1).getProp("sqlType"), equalTo("TIMESTAMP")); } + @Test + public void changeStreams() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", true); + Ddl ddl = + Ddl.builder() + .createChangeStream("ChangeStreamAll") + .forClause("FOR ALL") + .options( + ImmutableList.of( + "retention_period=\"7d\"", "value_capture_type=\"OLD_AND_NEW_VALUES\"")) + .endChangeStream() + .createChangeStream("ChangeStreamEmpty") + .endChangeStream() + .createChangeStream("ChangeStreamTableColumns") + .forClause("FOR `T1`, `T2`(`c1`, `c2`), `T3`()") + .endChangeStream() + .build(); + + Collection result = converter.convert(ddl); + assertThat(result, hasSize(3)); + for (Schema s : result) { + assertThat(s.getNamespace(), equalTo("spannertest")); + assertThat(s.getProp("googleFormatVersion"), equalTo("booleans")); + assertThat(s.getProp("googleStorage"), equalTo("CloudSpanner")); + assertThat(s.getFields(), empty()); + } + + Iterator it = result.iterator(); + Schema avroSchema1 = it.next(); + assertThat(avroSchema1.getName(), equalTo("ChangeStreamAll")); + assertThat(avroSchema1.getProp("spannerChangeStreamForClause"), equalTo("FOR ALL")); + assertThat(avroSchema1.getProp("spannerOption_0"), equalTo("retention_period=\"7d\"")); + assertThat( + avroSchema1.getProp("spannerOption_1"), + equalTo("value_capture_type=\"OLD_AND_NEW_VALUES\"")); + + Schema avroSchema2 = it.next(); + assertThat(avroSchema2.getName(), equalTo("ChangeStreamEmpty")); + assertThat(avroSchema2.getProp("spannerChangeStreamForClause"), equalTo("")); + assertThat(avroSchema2.getProp("spannerOption_0"), nullValue()); + + Schema avroSchema3 = it.next(); + assertThat(avroSchema3.getName(), equalTo("ChangeStreamTableColumns")); + assertThat( + avroSchema3.getProp("spannerChangeStreamForClause"), + equalTo("FOR `T1`, `T2`(`c1`, `c2`), `T3`()")); + assertThat(avroSchema3.getProp("spannerOption_0"), nullValue()); + } + private Schema nullableUnion(Schema.Type s) { return Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(s)); } From 0b723cc66e7f42ef525b69eff267defb0b233b74 Mon Sep 17 00:00:00 2001 From: dhercher Date: Sun, 10 Apr 2022 05:04:16 -0700 Subject: [PATCH 101/145] DatastreamToBigQuery for Oracle: Use BigQuery DATETIME datatype for Oracle dates to account for local timezone of Oracle DB in replication PiperOrigin-RevId: 440692687 --- .../cdc/mappers/BigQueryTableRowCleaner.java | 26 +++++++++++++++++-- .../teleport/v2/utils/DataStreamClient.java | 5 ++-- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryTableRowCleaner.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryTableRowCleaner.java index bb57ff0038..df645153a0 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryTableRowCleaner.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryTableRowCleaner.java @@ -53,6 +53,8 @@ public static void cleanTableRowField(TableRow row, FieldList tableFields, Strin cleanTableRowFieldStrings(row, tableFields, rowKey); } else if (fieldType == LegacySQLTypeName.DATE) { cleanTableRowFieldDates(row, tableFields, rowKey); + } else if (fieldType == LegacySQLTypeName.DATETIME) { + cleanTableRowFieldDateTime(row, tableFields, rowKey); } } @@ -80,8 +82,8 @@ public static void cleanTableRowFieldStrings(TableRow row, FieldList tableFields } /** - * Cleans the TableRow data for a given rowKey based on the requirements of a BigQuery String - * column type. + * Cleans the TableRow data for a given rowKey based on the requirements of a BigQuery DATE column + * type. * * @param row a TableRow object to clean. * @param tableFields a FieldList of Bigquery columns. @@ -99,4 +101,24 @@ public static void cleanTableRowFieldDates(TableRow row, FieldList tableFields, } } } + + /** + * Cleans the TableRow data for a given rowKey based on the requirements of a BigQuery DATETIME + * column type. + * + * @param row a TableRow object to clean. + * @param tableFields a FieldList of Bigquery columns. + * @param rowKey a String with the name of the field to clean. + */ + public static void cleanTableRowFieldDateTime( + TableRow row, FieldList tableFields, String rowKey) { + Object rowObject = row.get(rowKey); + if (rowObject instanceof String) { + String dateTimeString = (String) rowObject; + // Datetime types do not allow Z which resprents UTC timezone info + if (dateTimeString.endsWith("Z")) { + row.put(rowKey, dateTimeString.substring(0, dateTimeString.length() - 1)); + } + } + } } diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java index 5bb566a288..29466a1b18 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/utils/DataStreamClient.java @@ -140,7 +140,8 @@ public List getPrimaryKeys(String streamName, String schemaName, String } catch (IOException e) { if (e.toString() .contains( - "Quota exceeded for quota metric 'API requests' and limit 'API requests per minute' of service")) { + "Quota exceeded for quota metric 'API requests' and limit 'API requests per minute'" + + " of service")) { try { Thread.sleep(60 * 1000); return this.getPrimaryKeys(streamName, schemaName, tableName); @@ -386,7 +387,7 @@ public StandardSQLTypeName convertOracleToBigQueryColumnType(OracleColumn column case "LONG_RAW": return StandardSQLTypeName.BYTES; case "DATE": - return StandardSQLTypeName.DATE; + return StandardSQLTypeName.DATETIME; default: } From b90c6d9a404f592aa0efcc0965b46fa40655fc43 Mon Sep 17 00:00:00 2001 From: dhercher Date: Sun, 10 Apr 2022 09:03:23 -0700 Subject: [PATCH 102/145] DatastreamToBigQuery uses txn and row id for unqiue rollback value PiperOrigin-RevId: 440716577 --- .../com/google/cloud/teleport/v2/values/DatastreamRow.java | 4 ++++ .../cloud/teleport/v2/transforms/StatefulRowCleaner.java | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java index 59058c3eec..19b2420500 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java @@ -169,4 +169,8 @@ public String toString() { public String getOracleRowId() { return this.getStringValue(DEFAULT_ORACLE_PRIMARY_KEY); } + + public String getOracleTxnId() { + return this.getStringValue(ORACLE_TRANSACTION_ID_KEY); + } } diff --git a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java index 5556798452..47606ffe31 100644 --- a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java +++ b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/transforms/StatefulRowCleaner.java @@ -82,7 +82,7 @@ public void processElement(ProcessContext context) { TableRow tableRow = context.element(); DatastreamRow row = DatastreamRow.of(tableRow); if (row.getSourceType().equals("oracle")) { - context.output(KV.of(row.getOracleRowId(), tableRow)); + context.output(KV.of(getOracleRowKey(row), tableRow)); } else { String generatedString = RandomStringUtils.random( @@ -90,6 +90,10 @@ public void processElement(ProcessContext context) { context.output(KV.of(generatedString, tableRow)); } } + + private String getOracleRowKey(DatastreamRow row) { + return row.getOracleTxnId() + "#" + row.getOracleRowId(); + } } /** From 58a91af3e57f5597a4d56646e8305315df062ab3 Mon Sep 17 00:00:00 2001 From: dhercher Date: Tue, 12 Apr 2022 01:27:21 -0700 Subject: [PATCH 103/145] DatastreamToSQL template: bugfix for a rare case of multiple records loaded from Oracle having the same timestamp and SCN, potentially causing some valid new records to be considered stale and ignored. Template now uses Oracle's RS_ID and SSN in addition to timestamp and SCN when sorting records. DatastreamToBigQuery template: For Oracle sources, the Merge sort requires using rs_id in rare cases. This column will be added to tables and used in Merge queries PiperOrigin-RevId: 441113227 --- .../cloud/teleport/v2/cdc/mappers/BigQueryDefaultSchemas.java | 1 + .../java/com/google/cloud/teleport/v2/values/DatastreamRow.java | 2 +- .../cloud/teleport/v2/templates/DataStreamToBigQuery.java | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryDefaultSchemas.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryDefaultSchemas.java index e1559a3225..cc681dbb1d 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryDefaultSchemas.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/cdc/mappers/BigQueryDefaultSchemas.java @@ -52,6 +52,7 @@ public final class BigQueryDefaultSchemas { // Oracle specific metadata put("_metadata_row_id", StandardSQLTypeName.STRING); + put("_metadata_rs_id", StandardSQLTypeName.STRING); // MySQL Specific Metadata put("_metadata_log_file", StandardSQLTypeName.STRING); diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java index 19b2420500..ed14924bef 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/values/DatastreamRow.java @@ -153,7 +153,7 @@ public List getSortFields() { return Arrays.asList("_metadata_timestamp", "_metadata_log_file", "_metadata_log_position"); } else { // Current default is oracle. - return Arrays.asList("_metadata_timestamp", "_metadata_scn"); + return Arrays.asList("_metadata_timestamp", "_metadata_scn", "_metadata_rs_id"); } } diff --git a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java index e91472a219..ff49bbeef4 100644 --- a/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java +++ b/v2/datastream-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToBigQuery.java @@ -196,7 +196,7 @@ public interface Options "Fields to ignore in BigQuery (comma separator). eg. _metadata_stream,_metadata_schema") @Default.String( "_metadata_stream,_metadata_schema,_metadata_table,_metadata_source,_metadata_ssn," - + "_metadata_rs_id,_metadata_tx_id,_metadata_dlq_reconsumed,_metadata_primary_keys," + + "_metadata_tx_id,_metadata_dlq_reconsumed,_metadata_primary_keys," + "_metadata_error,_metadata_retry_count") String getIgnoreFields(); From 93dd3c0c44647f8254df4103309ab7814ea9c2ed Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Tue, 12 Apr 2022 07:53:11 -0700 Subject: [PATCH 104/145] Add doumentation for disabledAlgorithms parameter in Jdbc To BigQuery template. PiperOrigin-RevId: 441186957 --- .../google/cloud/teleport/templates/common/JdbcConverters.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java index 0060669e5f..f5428061e0 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/JdbcConverters.java @@ -97,7 +97,7 @@ public interface JdbcToBigQueryOptions extends PipelineOptions { void setKMSEncryptionKey(ValueProvider keyName); @Description( - "Comma seperated algorithms to disable. If this value is set to \"none\" then" + "Comma separated algorithms to disable. If this value is set to \"none\" then" + " jdk.tls.disabledAlgorithms is set to \"\". Use with care, as the algorithms" + " disabled by default are known to have either vulnerabilities or performance issues." + " for example: SSLv3, RC4.") From 9b8723fbdc705a872dc18e285fd2398ae79b4df2 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Thu, 21 Apr 2022 14:28:45 -0700 Subject: [PATCH 105/145] Implement Spanner Change Streams to BigQuery changelog table replication Dataflow template PiperOrigin-RevId: 443480253 --- ...SpannerChangeStreamsToBigQueryOptions.java | 127 +++++ .../BigQueryDeadLetterQueueSanitizer.java | 46 ++ .../BigQueryDynamicDestinations.java | 192 +++++++ .../FailsafeModJsonToTableRowTransformer.java | 270 ++++++++++ .../SpannerChangeStreamsToBigQuery.java | 368 ++++++++++++++ .../model/Mod.java | 258 ++++++++++ .../model/TrackedSpannerColumn.java | 38 ++ .../model/TrackedSpannerTable.java | 110 ++++ .../model/package-info.java | 18 + .../package-info.java | 18 + .../schemautils/BigQueryUtils.java | 86 ++++ .../schemautils/SpannerToBigQueryUtils.java | 235 +++++++++ .../schemautils/SpannerUtils.java | 360 +++++++++++++ .../schemautils/package-info.java | 18 + ...hangestreams-to-bigquery-command-spec.json | 7 + .../SchemaUtilsTest.java | 478 ++++++++++++++++++ 16 files changed, 2629 insertions(+) create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizer.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/Mod.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerColumn.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerTable.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/package-info.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/package-info.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/BigQueryUtils.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/package-info.java create mode 100644 v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-bigquery-command-spec.json create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java new file mode 100644 index 0000000000..026547abb9 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java @@ -0,0 +1,127 @@ +/* + * Copyright (C) 2022 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.v2.options; + +import com.google.cloud.spanner.Options.RpcPriority; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link SpannerChangeStreamsToBigQueryOptions} class provides the custom execution options + * passed by the executor at the command-line. + */ +public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineOptions { + + @Description("The Spanner instance ID that contains the Change Stream.") + @Validation.Required + String getSpannerInstanceId(); + + void setSpannerInstanceId(String value); + + @Description("The Spanner database ID that contains the Change Stream.") + @Validation.Required + String getSpannerDatabaseId(); + + void setSpannerDatabaseId(String value); + + @Description("The Spanner metadata instance ID that's used by the Change Stream connector.") + @Validation.Required + String getSpannerMetadataInstanceId(); + + void setSpannerMetadataInstanceId(String value); + + @Description("The Spanner metadata database ID that's used by the Change Stream connector.") + @Validation.Required + String getSpannerMetadataDatabaseId(); + + void setSpannerMetadataDatabaseId(String value); + + @Description("The name of the Spanner Change Stream.") + @Validation.Required + String getSpannerChangeStream(); + + void setSpannerChangeStream(String value); + + @Description( + "Priority for Spanner RPC invocations. Defaults to HIGH. Allowed priorites are LOW, MEDIUM," + + " HIGH.") + @Default.Enum("HIGH") + RpcPriority getSpannerRpcPriority(); + + void setSpannerRpcPriority(RpcPriority value); + + @Description("Spanner host endpoint (only used for testing).") + @Default.String("https://batch-spanner.googleapis.com") + String getSpannerHost(); + + void setSpannerHost(String value); + + @Description( + "The starting DateTime to use for reading Change Streams" + + " (https://tools.ietf.org/html/rfc3339). Defaults to pipeline start time.") + @Default.String("") + String getStartTimestamp(); + + void setStartTimestamp(String startTimestamp); + + @Description( + "The ending DateTime to use for reading Change Streams" + + " (https://tools.ietf.org/html/rfc3339). The default value is \"max\", which represents" + + " an infinite time in the future.") + @Default.String("") + String getEndTimestamp(); + + void setEndTimestamp(String startTimestamp); + + @Description("The output BigQuery dataset.") + @Validation.Required + String getBigQueryDataset(); + + void setBigQueryDataset(String value); + + @Description("The BigQuery Project ID. Default is the project for the Dataflow job.") + @Default.String("") + String getBigQueryProjectId(); + + void setBigQueryProjectId(String value); + + @Description("The changelog BigQuery table name Template") + @Default.String("{_metadata_spanner_table_name}_changelog") + String getBigQueryChangelogTableNameTemplate(); + + void setBigQueryChangelogTableNameTemplate(String value); + + @Description("The Dead Letter Queue GCS Prefix to use for errored data") + @Default.String("") + String getDlqDirectory(); + + void setDlqDirectory(String value); + + @Description("The number of minutes between deadletter queue retries") + @Default.Integer(10) + Integer getDlqRetryMinutes(); + + void setDlqRetryMinutes(Integer value); + + // TODO(haikuo-google): Implement this functionality. + @Description("Comma Separated list of metadata fields to be ignored") + @Default.String("") + String getIgnoreMetadataFields(); + + void setIgnoreMetadataFields(String value); +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizer.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizer.java new file mode 100644 index 0000000000..257b953338 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizer.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.teleport.v2.cdc.dlq.DeadLetterQueueSanitizer; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryInsertError; + +/** + * Class {@link BigQueryDeadLetterQueueSanitizer} cleans and prepares failed BigQuery inserts to be + * stored in a GCS Dead Letter Queue. NOTE: The input to a Sanitizer is flexible but the output must + * be a String unless your override formatMessage(). + */ +public final class BigQueryDeadLetterQueueSanitizer + extends DeadLetterQueueSanitizer { + + public BigQueryDeadLetterQueueSanitizer() {} + + @Override + public String getJsonMessage(BigQueryInsertError input) { + TableRow tableRow = input.getRow(); + /** Extract the original payload from the {@link TableRow}. */ + String message = + (String) tableRow.get(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_ORIGINAL_PAYLOAD_JSON); + return message; + } + + @Override + public String getErrorMessageJson(BigQueryInsertError input) { + return input.getError().toString(); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java new file mode 100644 index 0000000000..515a245848 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java @@ -0,0 +1,192 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.auto.value.AutoValue; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerTable; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; +import com.google.cloud.teleport.v2.transforms.BigQueryConverters; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.TableDestination; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.ValueInSingleWindow; + +/** + * The {@link BigQueryDynamicDestinations} loads into BigQuery tables in a dynamic fashion. The + * destination table is inferred from the provided {@link TableRow}. + */ +public final class BigQueryDynamicDestinations + extends DynamicDestinations> { + + private final Map spannerTableByName; + private final String bigQueryProject, bigQueryDataset, bigQueryTableTemplate; + + public static BigQueryDynamicDestinations of( + BigQueryDynamicDestinationsOptions bigQueryDynamicDestinationsOptions) { + try (SpannerAccessor spannerAccessor = + SpannerAccessor.getOrCreate(bigQueryDynamicDestinationsOptions.getSpannerConfig())) { + Map spannerTableByName = + new SpannerUtils( + spannerAccessor.getDatabaseClient(), + bigQueryDynamicDestinationsOptions.getChangeStreamName()) + .getSpannerTableByName(); + return new BigQueryDynamicDestinations( + bigQueryDynamicDestinationsOptions, spannerTableByName); + } + } + + private BigQueryDynamicDestinations( + BigQueryDynamicDestinationsOptions bigQueryDynamicDestinationsOptions, + Map spannerTableByName) { + this.spannerTableByName = spannerTableByName; + this.bigQueryProject = bigQueryDynamicDestinationsOptions.getBigQueryProject(); + this.bigQueryDataset = bigQueryDynamicDestinationsOptions.getBigQueryDataset(); + this.bigQueryTableTemplate = bigQueryDynamicDestinationsOptions.getBigQueryTableTemplate(); + } + + private TableId getTableId(String bigQueryTableTemplate, TableRow tableRow) { + String bigQueryTableName = + BigQueryConverters.formatStringTemplate(bigQueryTableTemplate, tableRow); + + return TableId.of(bigQueryProject, bigQueryDataset, bigQueryTableName); + } + + @Override + public KV getDestination(ValueInSingleWindow element) { + TableRow tableRow = element.getValue(); + return KV.of(getTableId(bigQueryTableTemplate, tableRow), tableRow); + } + + @Override + public TableDestination getTable(KV destination) { + TableId tableId = getTableId(bigQueryTableTemplate, destination.getValue()); + String tableName = + String.format("%s:%s.%s", tableId.getProject(), tableId.getDataset(), tableId.getTable()); + + return new TableDestination(tableName, "BigQuery changelog table."); + } + + @Override + public TableSchema getSchema(KV destination) { + TableRow tableRow = destination.getValue(); + String spannerTableName = + (String) tableRow.get(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_TABLE_NAME); + TrackedSpannerTable spannerTable = spannerTableByName.get(spannerTableName); + + List fields = + SpannerToBigQueryUtils.spannerColumnsToBigQueryIOFields(spannerTable.getAllColumns()); + + // Add all metadata fields. + String requiredMode = Field.Mode.REQUIRED.name(); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_MOD_TYPE) + .setType(StandardSQLTypeName.STRING.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_TABLE_NAME) + .setType(StandardSQLTypeName.STRING.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_SPANNER_COMMIT_TIMESTAMP) + .setType(StandardSQLTypeName.TIMESTAMP.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_SERVER_TRANSACTION_ID) + .setType(StandardSQLTypeName.STRING.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_RECORD_SEQUENCE) + .setType(StandardSQLTypeName.STRING.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName( + BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_IS_LAST_RECORD_IN_TRANSACTION_IN_PARTITION) + .setType(StandardSQLTypeName.BOOL.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_NUMBER_OF_RECORDS_IN_TRANSACTION) + .setType(StandardSQLTypeName.INT64.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_NUMBER_OF_PARTITIONS_IN_TRANSACTION) + .setType(StandardSQLTypeName.INT64.name()) + .setMode(requiredMode)); + fields.add( + new TableFieldSchema() + .setName(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_BIGQUERY_COMMIT_TIMESTAMP) + .setType(StandardSQLTypeName.TIMESTAMP.name()) + .setMode(requiredMode)); + + return new TableSchema().setFields(fields); + } + + /** + * {@link BigQueryDynamicDestinationsOptions} provides options to initialize {@link + * BigQueryDynamicDestinations}. + */ + @AutoValue + public abstract static class BigQueryDynamicDestinationsOptions implements Serializable { + public abstract SpannerConfig getSpannerConfig(); + + public abstract String getChangeStreamName(); + + public abstract String getBigQueryProject(); + + public abstract String getBigQueryDataset(); + + public abstract String getBigQueryTableTemplate(); + + static Builder builder() { + return new AutoValue_BigQueryDynamicDestinations_BigQueryDynamicDestinationsOptions.Builder(); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setSpannerConfig(SpannerConfig spannerConfig); + + abstract Builder setChangeStreamName(String changeStreamName); + + abstract Builder setBigQueryProject(String bigQueryProject); + + abstract Builder setBigQueryDataset(String bigQueryDataset); + + abstract Builder setBigQueryTableTemplate(String bigQueryTableTemplate); + + abstract BigQueryDynamicDestinationsOptions build(); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java new file mode 100644 index 0000000000..0da3be3516 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java @@ -0,0 +1,270 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Key.Builder; +import com.google.cloud.spanner.KeySet; +import com.google.cloud.spanner.Options; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.TimestampBound; +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.Mod; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerColumn; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerTable; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ModType; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.node.ObjectNode; +import org.json.JSONObject; + +/** + * Class {@link FailsafeModJsonToTableRowTransformer} provides methods that convert a {@link Mod} + * JSON string wrapped in {@link FailsafeElement} to a {@link TableRow}. + */ +public final class FailsafeModJsonToTableRowTransformer { + + /** + * Primary class for taking a {@link FailsafeElement} {@link Mod} JSON input and converting to a + * {@link TableRow}. + */ + public static class FailsafeModJsonToTableRow + extends PTransform>, PCollectionTuple> { + + /** The tag for the main output of the transformation. */ + public TupleTag transformOut = new TupleTag() {}; + + /** The tag for the dead letter output of the transformation. */ + public TupleTag> transformDeadLetterOut = + new TupleTag>() {}; + + private final FailsafeModJsonToTableRowOptions failsafeModJsonToTableRowOptions; + + public FailsafeModJsonToTableRow( + FailsafeModJsonToTableRowOptions failsafeModJsonToTableRowOptions) { + this.failsafeModJsonToTableRowOptions = failsafeModJsonToTableRowOptions; + } + + public PCollectionTuple expand(PCollection> input) { + PCollectionTuple out = + input.apply( + ParDo.of( + new FailsafeModJsonToTableRowFn( + failsafeModJsonToTableRowOptions.getSpannerConfig(), + failsafeModJsonToTableRowOptions.getSpannerChangeStream(), + transformOut, + transformDeadLetterOut)) + .withOutputTags(transformOut, TupleTagList.of(transformDeadLetterOut))); + out.get(transformDeadLetterOut).setCoder(failsafeModJsonToTableRowOptions.getCoder()); + return out; + } + + /** + * The {@link FailsafeModJsonToTableRowFn} converts a {@link Mod} JSON string wrapped in {@link + * FailsafeElement} to a {@link TableRow}. + */ + public static class FailsafeModJsonToTableRowFn + extends DoFn, TableRow> { + + private transient SpannerAccessor spannerAccessor; + private final SpannerConfig spannerConfig; + private final String spannerChangeStream; + private Map spannerTableByName = null; + public TupleTag transformOut; + public TupleTag> transformDeadLetterOut; + + public FailsafeModJsonToTableRowFn( + SpannerConfig spannerConfig, + String spannerChangeStream, + TupleTag transformOut, + TupleTag> transformDeadLetterOut) { + this.spannerConfig = spannerConfig; + this.spannerChangeStream = spannerChangeStream; + this.transformOut = transformOut; + this.transformDeadLetterOut = transformDeadLetterOut; + } + + @Setup + public void setUp() { + spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); + spannerTableByName = + new SpannerUtils(spannerAccessor.getDatabaseClient(), spannerChangeStream) + .getSpannerTableByName(); + } + + @Teardown + public void tearDown() { + spannerAccessor.close(); + } + + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement failsafeModJsonString = context.element(); + + try { + TableRow tableRow = modJsonStringToTableRow(failsafeModJsonString.getPayload()); + context.output(tableRow); + } catch (Exception e) { + context.output( + transformDeadLetterOut, + FailsafeElement.of(failsafeModJsonString) + .setErrorMessage(e.getMessage()) + .setStacktrace(Throwables.getStackTraceAsString(e))); + } + } + + private TableRow modJsonStringToTableRow(String modJsonString) throws Exception { + ObjectNode modObjectNode = (ObjectNode) new ObjectMapper().readTree(modJsonString); + for (String excludeFieldName : BigQueryUtils.getBigQueryIntermediateMetadataFieldNames()) { + if (modObjectNode.has(excludeFieldName)) { + modObjectNode.remove(excludeFieldName); + } + } + + Mod mod = Mod.fromJson(modObjectNode.toString()); + String spannerTableName = mod.getTableName(); + TrackedSpannerTable spannerTable = spannerTableByName.get(spannerTableName); + com.google.cloud.Timestamp spannerCommitTimestamp = + com.google.cloud.Timestamp.ofTimeSecondsAndNanos( + mod.getCommitTimestampSeconds(), mod.getCommitTimestampNanos()); + + // Set metadata fields of the tableRow. + TableRow tableRow = new TableRow(); + BigQueryUtils.setMetadataFiledsOfTableRow( + spannerTableName, mod, modJsonString, spannerCommitTimestamp, tableRow); + JSONObject keysJsonObject = new JSONObject(mod.getKeysJson()); + // Set Spanner key columns of of the tableRow. + for (TrackedSpannerColumn spannerColumn : spannerTable.getPkColumns()) { + String spannerColumnName = spannerColumn.getName(); + if (keysJsonObject.has(spannerColumnName)) { + tableRow.set(spannerColumnName, keysJsonObject.get(spannerColumnName)); + } else { + throw new IllegalArgumentException( + "Cannot find value for key column " + spannerColumnName); + } + } + + // For "DELETE" mod, we only need to set the key columns. + if (mod.getModType() == ModType.DELETE) { + return tableRow; + } + + // Set non-key columns of the tableRow. + SpannerToBigQueryUtils.addSpannerNonPkColumnsToTableRow( + mod.getNewValuesJson(), spannerTable.getNonPkColumns(), tableRow); + + // For "INSERT" mod, we can get all columns from mod. + if (mod.getModType() == ModType.INSERT) { + return tableRow; + } + + // For "UPDATE" mod, the Mod only contains the changed columns, unchanged tracked columns + // are not included, so we need to do a snapshot read to Spanner to get the full row image + // tracked by change stream, we want to re-read the updated columns as well to get a + // consistent view of the whole row after the transaction is committed. + // Note that the read can fail if the database version retention period (default to be one + // hour) has passed the snapshot read timestamp, similar to other error cases, the pipeline + // will put the failed mod into the retry deadletter queue, and retry it for 5 times, and + // then eventually add the failed mod into the severe deadletter queue which won't be + // processed by the pipeline again, users should process the severe deadletter queue + // themselves. + Builder keyBuilder = com.google.cloud.spanner.Key.newBuilder(); + for (TrackedSpannerColumn spannerColumn : spannerTable.getPkColumns()) { + String spannerColumnName = spannerColumn.getName(); + if (keysJsonObject.has(spannerColumnName)) { + SpannerUtils.appendToSpannerKey(spannerColumn, keysJsonObject, keyBuilder); + } else { + throw new IllegalArgumentException( + "Cannot find value for key column " + spannerColumnName); + } + } + + List spannerNonPkColumns = spannerTable.getNonPkColumns(); + List spannerNonPkColumnNames = + spannerNonPkColumns.stream() + .map(spannerNonPkColumn -> spannerNonPkColumn.getName()) + .collect(Collectors.toList()); + + Options.ReadQueryUpdateTransactionOption options = + Options.priority(spannerConfig.getRpcPriority().get()); + // We assume the Spanner schema isn't changed while the pipeline is running, so the read is + // expected to succeed in normal cases. The schema change is currently not supported. + try (ResultSet resultSet = + spannerAccessor + .getDatabaseClient() + .singleUseReadOnlyTransaction( + TimestampBound.ofReadTimestamp(spannerCommitTimestamp)) + .read( + spannerTable.getTableName(), + KeySet.singleKey(keyBuilder.build()), + spannerNonPkColumnNames, + options)) { + SpannerToBigQueryUtils.spannerSnapshotRowToBigQueryTableRow( + resultSet, spannerNonPkColumns, tableRow); + } + + return tableRow; + } + } + } + + /** + * {@link FailsafeModJsonToTableRowOptions} provides options to initialize {@link + * FailsafeModJsonToTableRowTransformer}. + */ + @AutoValue + public abstract static class FailsafeModJsonToTableRowOptions implements Serializable { + public abstract SpannerConfig getSpannerConfig(); + + public abstract String getSpannerChangeStream(); + + public abstract FailsafeElementCoder getCoder(); + + static Builder builder() { + return new AutoValue_FailsafeModJsonToTableRowTransformer_FailsafeModJsonToTableRowOptions + .Builder(); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setSpannerConfig(SpannerConfig spannerSpannerConfig); + + abstract Builder setSpannerChangeStream(String spannerChangeStream); + + abstract Builder setCoder(FailsafeElementCoder coder); + + abstract FailsafeModJsonToTableRowOptions build(); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java new file mode 100644 index 0000000000..04287ff92b --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java @@ -0,0 +1,368 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.Timestamp; +import com.google.cloud.teleport.v2.cdc.dlq.DeadLetterQueueManager; +import com.google.cloud.teleport.v2.cdc.dlq.StringDeadLetterQueueSanitizer; +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; +import com.google.cloud.teleport.v2.options.SpannerChangeStreamsToBigQueryOptions; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.Mod; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import com.google.cloud.teleport.v2.transforms.DLQWriteTransform; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy; +import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; +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.sdk.values.PCollectionTuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// TODO(haikuo-google): Add integration test. +// TODO(haikuo-google): Add README. +// TODO(haikuo-google): Add stackdriver metrics. +// TODO(haikuo-google): Ideally side input should be used to store schema information and shared +// accrss DoFns, but since side input fix is not yet deployed at the moment, we read schema +// information in the beginning of the DoFn as a work around. We should use side input instead when +// it's available. +// TODO(haikuo-google): Test the case where tables or columns are added while the pipeline is +// running. +/** + * This pipeline ingests {@link DataChangeRecord} from Spanner change stream. The {@link + * DataChangeRecord} is then broken into {@link Mod}, which converted into {@link TableRow} and + * inserted into BigQuery table. + */ +public final class SpannerChangeStreamsToBigQuery { + + /** String/String Coder for {@link FailsafeElement}. */ + public static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + private static final Logger LOG = LoggerFactory.getLogger(SpannerChangeStreamsToBigQuery.class); + + // Max number of deadletter queue retries. + private static final int DLQ_MAX_RETRIES = 5; + + private static final String USE_RUNNER_V2_EXPERIMENT = "use_runner_v2"; + + /** + * Main entry point for executing the pipeline. + * + * @param args The command-line arguments to the pipeline. + */ + public static void main(String[] args) { + LOG.info("Starting to replicate change records from Spanner change streams to BigQuery"); + + SpannerChangeStreamsToBigQueryOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(SpannerChangeStreamsToBigQueryOptions.class); + + run(options); + } + + private static void validateOptions(SpannerChangeStreamsToBigQueryOptions options) { + if (options.getDlqRetryMinutes() <= 0) { + throw new IllegalArgumentException("dlqRetryMinutes must be positive."); + } + } + + private static void setOptions(SpannerChangeStreamsToBigQueryOptions options) { + options.setStreaming(true); + options.setEnableStreamingEngine(true); + // TODO(haikuo-google): Enable scaling when it's supported. + options.setAutoscalingAlgorithm( + DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE); + + // Add use_runner_v2 to the experiments option, since change streams connector is only supported + // on Dataflow runner v2. + List experiments = options.getExperiments(); + if (experiments == null) { + experiments = new ArrayList<>(); + } + boolean hasUseRunnerV2 = false; + for (String experiment : experiments) { + if (experiment.toLowerCase().equals(USE_RUNNER_V2_EXPERIMENT)) { + hasUseRunnerV2 = true; + break; + } + } + if (!hasUseRunnerV2) { + experiments.add(USE_RUNNER_V2_EXPERIMENT); + } + options.setExperiments(experiments); + } + + /** + * Runs the pipeline with the supplied options. + * + * @param options The execution parameters to the pipeline. + * @return The result of the pipeline execution. + */ + public static PipelineResult run(SpannerChangeStreamsToBigQueryOptions options) { + setOptions(options); + validateOptions(options); + + /** + * Stages: 1) Read {@link DataChangeRecord} from change stream. 2) Create {@link + * FailsafeElement} of {@link Mod} JSON and merge from: - {@link DataChangeRecord}. - GCS Dead + * letter queue. 3) Convert {@link Mod} JSON into {@link TableRow} by reading from Spanner at + * commit timestamp. 4) Append {@link TableRow} to BigQuery. 5) Write Failures from 2), 3) and + * 4) to GCS dead letter queue. + */ + Pipeline pipeline = Pipeline.create(options); + DeadLetterQueueManager dlqManager = buildDlqManager(options); + + String dlqDirectory = dlqManager.getRetryDlqDirectoryWithDateTime(); + String tempDlqDirectory = dlqManager.getRetryDlqDirectory() + "tmp/"; + + // Retrieve and parse the startTimestamp and endTimestamp. + Timestamp startTimestamp = + options.getStartTimestamp().isEmpty() + ? Timestamp.now() + : Timestamp.parseTimestamp(options.getStartTimestamp()); + Timestamp endTimestamp = + options.getEndTimestamp().isEmpty() + ? Timestamp.MAX_VALUE + : Timestamp.parseTimestamp(options.getEndTimestamp()); + + SpannerConfig spannerConfig = + SpannerConfig.create() + .withHost(ValueProvider.StaticValueProvider.of(options.getSpannerHost())) + .withProjectId(options.getProject()) + .withInstanceId(options.getSpannerInstanceId()) + .withDatabaseId(options.getSpannerDatabaseId()) + .withRpcPriority(options.getSpannerRpcPriority()); + + SpannerIO.ReadChangeStream readChangeStream = + SpannerIO.readChangeStream() + .withSpannerConfig(spannerConfig) + .withMetadataInstance(options.getSpannerMetadataInstanceId()) + .withMetadataDatabase(options.getSpannerMetadataDatabaseId()) + .withChangeStreamName(options.getSpannerChangeStream()) + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp) + .withRpcPriority(options.getSpannerRpcPriority()); + + PCollection dataChangeRecord = + pipeline + .apply("Read from Spanner Change Streams", readChangeStream) + .apply("Reshuffle DataChangeRecord", Reshuffle.viaRandomKey()); + + PCollection> sourceFailsafeModJson = + dataChangeRecord + .apply("DataChangeRecord To Mod JSON", ParDo.of(new DataChangeRecordToModJsonFn())) + .apply( + "Wrap Mod JSON In FailsafeElement", + ParDo.of( + new DoFn>() { + @ProcessElement + public void process( + @Element String input, + OutputReceiver> receiver) { + receiver.output(FailsafeElement.of(input, input)); + } + })) + .setCoder(FAILSAFE_ELEMENT_CODER); + + PCollectionTuple dlqModJson = + dlqManager.getReconsumerDataTransform( + pipeline.apply(dlqManager.dlqReconsumer(options.getDlqRetryMinutes()))); + PCollection> retryableDlqFailsafeModJson = + dlqModJson.get(DeadLetterQueueManager.RETRYABLE_ERRORS).setCoder(FAILSAFE_ELEMENT_CODER); + + PCollection> failsafeModJson = + PCollectionList.of(sourceFailsafeModJson) + .and(retryableDlqFailsafeModJson) + .apply("Merge Source And DLQ Mod JSON", Flatten.pCollections()); + + FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions + failsafeModJsonToTableRowOptions = + FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions.builder() + .setSpannerConfig(spannerConfig) + .setSpannerChangeStream(options.getSpannerChangeStream()) + .setCoder(FAILSAFE_ELEMENT_CODER) + .build(); + FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow failsafeModJsonToTableRow = + new FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow( + failsafeModJsonToTableRowOptions); + + PCollectionTuple tableRowTuple = + failsafeModJson.apply("Mod JSON To TableRow", failsafeModJsonToTableRow); + + BigQueryDynamicDestinations.BigQueryDynamicDestinationsOptions + bigQueryDynamicDestinationsOptions = + BigQueryDynamicDestinations.BigQueryDynamicDestinationsOptions.builder() + .setSpannerConfig(spannerConfig) + .setChangeStreamName(options.getSpannerChangeStream()) + .setBigQueryProject(getBigQueryProject(options)) + .setBigQueryDataset(options.getBigQueryDataset()) + .setBigQueryTableTemplate(options.getBigQueryChangelogTableNameTemplate()) + .build(); + WriteResult writeResult = + tableRowTuple + .get(failsafeModJsonToTableRow.transformOut) + .apply( + "Write To BigQuery", + BigQueryIO.write() + .to(BigQueryDynamicDestinations.of(bigQueryDynamicDestinationsOptions)) + .withFormatFunction(element -> removeIntermediateMetadataFields(element)) + .withFormatRecordOnFailureFunction(element -> element) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(Write.WriteDisposition.WRITE_APPEND) + .withExtendedErrorInfo() + .withMethod(Write.Method.STREAMING_INSERTS) + .withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors())); + + PCollection transformDlqJson = + tableRowTuple + .get(failsafeModJsonToTableRow.transformDeadLetterOut) + .apply( + "Failed Mod JSON During Table Row Transformation", + MapElements.via(new StringDeadLetterQueueSanitizer())); + + PCollection bqWriteDlqJson = + writeResult + .getFailedInsertsWithErr() + .apply( + "Failed Mod JSON During BigQuery Writes", + MapElements.via(new BigQueryDeadLetterQueueSanitizer())); + + PCollectionList.of(transformDlqJson) + .and(bqWriteDlqJson) + .apply("Merge Failed Mod JSON From Transform And BigQuery", Flatten.pCollections()) + .apply( + "Write Failed Mod JSON To DLQ", + DLQWriteTransform.WriteDLQ.newBuilder() + .withDlqDirectory(dlqDirectory) + .withTmpDirectory(tempDlqDirectory) + .build()); + + PCollection> nonRetryableDlqModJsonFailsafe = + dlqModJson.get(DeadLetterQueueManager.PERMANENT_ERRORS).setCoder(FAILSAFE_ELEMENT_CODER); + + nonRetryableDlqModJsonFailsafe + .apply( + "Write Mod JSON With Non-retryable Error To DLQ", + MapElements.via(new StringDeadLetterQueueSanitizer())) + .setCoder(StringUtf8Coder.of()) + .apply( + DLQWriteTransform.WriteDLQ.newBuilder() + .withDlqDirectory(dlqManager.getSevereDlqDirectoryWithDateTime()) + .withTmpDirectory(dlqManager.getSevereDlqDirectory() + "tmp/") + .build()); + + return pipeline.run(); + } + + private static DeadLetterQueueManager buildDlqManager( + SpannerChangeStreamsToBigQueryOptions options) { + String tempLocation = + options.as(DataflowPipelineOptions.class).getTempLocation().endsWith("/") + ? options.as(DataflowPipelineOptions.class).getTempLocation() + : options.as(DataflowPipelineOptions.class).getTempLocation() + "/"; + + String dlqDirectory = + options.getDlqDirectory().isEmpty() ? tempLocation + "dlq/" : options.getDlqDirectory(); + + LOG.info("Dead letter queue directory: {}", dlqDirectory); + return DeadLetterQueueManager.create(dlqDirectory, DLQ_MAX_RETRIES); + } + + private static String getBigQueryProject(SpannerChangeStreamsToBigQueryOptions options) { + return options.getBigQueryProjectId().isEmpty() + ? options.getProject() + : options.getBigQueryProjectId(); + } + + /** + * Remove the following intermediate metadata fields that are not user data from {@link TableRow}: + * _metadata_error, _metadata_retry_count, _metadata_spanner_original_payload_json. + */ + private static TableRow removeIntermediateMetadataFields(TableRow tableRow) { + TableRow cleanTableRow = tableRow.clone(); + Set rowKeys = tableRow.keySet(); + Set metadataFields = BigQueryUtils.getBigQueryIntermediateMetadataFieldNames(); + + for (String rowKey : rowKeys) { + if (metadataFields.contains(rowKey)) { + cleanTableRow.remove(rowKey); + } + } + + return cleanTableRow; + } + + /** + * DoFn that converts a {@link DataChangeRecord} to multiple {@link Mod} in serialized JSON + * format. + */ + static class DataChangeRecordToModJsonFn extends DoFn { + + @ProcessElement + public void process(@Element DataChangeRecord input, OutputReceiver receiver) { + for (org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod changeStreamsMod : + input.getMods()) { + Mod mod = + new Mod( + changeStreamsMod.getKeysJson(), + changeStreamsMod.getNewValuesJson(), + input.getCommitTimestamp(), + input.getServerTransactionId(), + input.isLastRecordInTransactionInPartition(), + input.getRecordSequence(), + input.getTableName(), + input.getModType(), + input.getNumberOfRecordsInTransaction(), + input.getNumberOfPartitionsInTransaction()); + + String modJsonString; + + try { + modJsonString = mod.toJson(); + } catch (IOException e) { + // Ignore exception and print bad format. + modJsonString = String.format("\"%s\"", input); + } + receiver.output(modJsonString); + } + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/Mod.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/Mod.java new file mode 100644 index 0000000000..aee4e70449 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/Mod.java @@ -0,0 +1,258 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery.model; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.cloud.Timestamp; +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ModType; + +/** + * The {@link Mod} contains the keys, new values (from {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod}) and metadata ({@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}) of a Spanner row. Note + * it's different from the {@link org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod}. + */ +@DefaultCoder(AvroCoder.class) +public final class Mod implements Serializable { + + private static final long serialVersionUID = 8703257194338184299L; + + private String keysJson; + private String newValuesJson; + private long commitTimestampSeconds; + private int commitTimestampNanos; + private String serverTransactionId; + private boolean isLastRecordInTransactionInPartition; + private String recordSequence; + private String tableName; + private ModType modType; + private long numberOfRecordsInTransaction; + private long numberOfPartitionsInTransaction; + + /** Default constructor for serialization only. */ + private Mod() {} + + /** + * @param keysJson JSON object as String, where the keys are the primary key column names and the + * values are the primary key column values + * @param newValuesJson JSON object as String, displaying the new state of the columns modified. + * This JSON object can be null in the case of a DELETE + * @param commitTimestamp the timestamp at which the modifications within were committed in Cloud + * Spanner + * @param serverTransactionId the unique transaction id in which the modifications occurred + * @param isLastRecordInTransactionInPartition indicates whether this record is the last emitted + * for the given transaction in the given partition + * @param recordSequence indicates the order in which this modification was made in the original + * Spanner transaction. The value is unique and monotonically increasing in the context of a + * particular serverTransactionId + * @param tableName the name of the table in which the modifications occurred + * @param modType the operation that caused the modification to occur, can only be one of INSERT, + * UPDATE and DELETE + * @param numberOfRecordsInTransaction the total number of records for the given transaction + * @param numberOfPartitionsInTransaction the total number of partitions within the given + * transaction + * @return a {@link Mod} corresponding to a {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod}. The constructed {@link Mod} is + * used as the processing unit of the pipeline, it contains all the information from {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord} and {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.Mod}, except columns JSON, since we + * will read the columns from Spanner instead. + */ + public Mod( + String keysJson, + String newValuesJson, + Timestamp commitTimestamp, + String serverTransactionId, + boolean isLastRecordInTransactionInPartition, + String recordSequence, + String tableName, + ModType modType, + long numberOfRecordsInTransaction, + long numberOfPartitionsInTransaction) { + this.keysJson = keysJson; + this.newValuesJson = newValuesJson; + this.commitTimestampSeconds = commitTimestamp.getSeconds(); + this.commitTimestampNanos = commitTimestamp.getNanos(); + this.serverTransactionId = serverTransactionId; + this.isLastRecordInTransactionInPartition = isLastRecordInTransactionInPartition; + this.recordSequence = recordSequence; + this.tableName = tableName; + this.modType = modType; + this.numberOfRecordsInTransaction = numberOfRecordsInTransaction; + this.numberOfPartitionsInTransaction = numberOfPartitionsInTransaction; + } + + public static Mod fromJson(String json) throws IOException { + return new ObjectMapper().readValue(json, Mod.class); + } + + /** + * The primary keys of this specific modification. This is always present and can not be null. The + * keys are returned as a JSON object (stringified), where the keys are the column names and the + * values are the column values. + * + * @return JSON object as String representing the primary key state for the row modified + */ + public String getKeysJson() { + return keysJson; + } + + /** + * The new column values after the modification was applied. This can be null when the + * modification was emitted for a DELETE operation. The values are returned as a JSON object + * (stringified), where the keys are the column names and the values are the column values. + * + * @return JSON object as String representing the new column values after the row was modified + */ + public @Nullable String getNewValuesJson() { + return newValuesJson; + } + + /** + * The seconds part of the timestamp at which the modifications within were committed in Cloud + * Spanner. + */ + public long getCommitTimestampSeconds() { + return commitTimestampSeconds; + } + + /** + * The nanoseconds part of the timestamp at which the modifications within were committed in Cloud + * Spanner. + */ + public int getCommitTimestampNanos() { + return commitTimestampNanos; + } + + /** The unique transaction id in which the modifications occurred. */ + public String getServerTransactionId() { + return serverTransactionId; + } + + /** + * Indicates whether this record is the last emitted for the given transaction in the given + * partition. + */ + public boolean getIsLastRecordInTransactionInPartition() { + return isLastRecordInTransactionInPartition; + } + + /** + * indicates the order in which this modification was made in the original Spanner transaction. + * The value is unique and monotonically increasing in the context of a particular + * serverTransactionId. + */ + public String getRecordSequence() { + return recordSequence; + } + + /** The name of the table in which the modifications within this record occurred. */ + public String getTableName() { + return tableName; + } + + /** The type of operation that caused the modifications within this record. */ + public ModType getModType() { + return modType; + } + + /** The total number of data change records for the given transaction. */ + public long getNumberOfRecordsInTransaction() { + return numberOfRecordsInTransaction; + } + + /** The total number of partitions for the given transaction. */ + public long getNumberOfPartitionsInTransaction() { + return numberOfPartitionsInTransaction; + } + + @Override + public boolean equals(@javax.annotation.Nullable Object o) { + if (this == o) { + return true; + } + if (!(o instanceof Mod)) { + return false; + } + Mod that = (Mod) o; + return keysJson == that.keysJson + && isLastRecordInTransactionInPartition == that.isLastRecordInTransactionInPartition + && numberOfRecordsInTransaction == that.numberOfRecordsInTransaction + && numberOfPartitionsInTransaction == that.numberOfPartitionsInTransaction + && commitTimestampSeconds == that.commitTimestampSeconds + && commitTimestampNanos == that.commitTimestampNanos + && Objects.equals(serverTransactionId, that.serverTransactionId) + && Objects.equals(recordSequence, that.recordSequence) + && Objects.equals(tableName, that.tableName) + && modType == that.modType; + } + + @Override + public int hashCode() { + return Objects.hash( + keysJson, + commitTimestampSeconds, + commitTimestampNanos, + serverTransactionId, + isLastRecordInTransactionInPartition, + recordSequence, + tableName, + modType, + numberOfRecordsInTransaction, + numberOfPartitionsInTransaction); + } + + @Override + public String toString() { + return "Mod{" + + "keysJson='" + + keysJson + + '\'' + + ", commitTimestampSeconds=" + + commitTimestampSeconds + + ", commitTimestampNanos=" + + commitTimestampNanos + + ", serverTransactionId='" + + serverTransactionId + + '\'' + + ", isLastRecordInTransactionInPartition=" + + isLastRecordInTransactionInPartition + + ", recordSequence='" + + recordSequence + + '\'' + + ", tableName='" + + tableName + + '\'' + + ", modType=" + + modType + + ", numberOfRecordsInTransaction=" + + numberOfRecordsInTransaction + + ", numberOfPartitionsInTransaction=" + + numberOfPartitionsInTransaction + + '}'; + } + + public String toJson() throws JsonProcessingException { + return new ObjectMapper().writeValueAsString(this); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerColumn.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerColumn.java new file mode 100644 index 0000000000..201d905ab6 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerColumn.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery.model; + +import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Type; +import java.io.Serializable; + +/** + * The {@link TrackedSpannerColumn} contains the name, type and ordinal position of a Spanner column + * tracked by a change stream. + */ +@AutoValue +public abstract class TrackedSpannerColumn implements Serializable { + + public static TrackedSpannerColumn create(String name, Type type, int ordinalPosition) { + return new AutoValue_TrackedSpannerColumn(name, type, ordinalPosition); + } + + public abstract String getName(); + + public abstract Type getType(); + + public abstract int getOrdinalPosition(); +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerTable.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerTable.java new file mode 100644 index 0000000000..a62f12a10c --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/TrackedSpannerTable.java @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery.model; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; + +/** + * The {@link TrackedSpannerTable} contains the table name and the columns of a Spanner table + * tracked by a change stream. + */ +@DefaultCoder(AvroCoder.class) +public final class TrackedSpannerTable implements Serializable { + + private String tableName; + // Primary key should be exactly the same as the tracked Spanner table. + private List pkColumns; + // Non-primary key only include the tracked Spanner columns. + private List nonPkColumns; + private List allColumns; + + /** Default constructor for serialization only. */ + public TrackedSpannerTable() {} + + private static class SortByOrdinalPosition implements Comparator { + public int compare(TrackedSpannerColumn o1, TrackedSpannerColumn o2) { + return Integer.compare(o1.getOrdinalPosition(), o2.getOrdinalPosition()); + } + } + + public TrackedSpannerTable( + String tableName, + List pkColumns, + List nonPkColumns) { + this.pkColumns = new ArrayList<>(pkColumns); + this.nonPkColumns = new ArrayList<>(nonPkColumns); + Collections.sort(this.pkColumns, new SortByOrdinalPosition()); + Collections.sort(this.nonPkColumns, new SortByOrdinalPosition()); + this.tableName = tableName; + + this.allColumns = new ArrayList<>(this.pkColumns.size() + this.nonPkColumns.size()); + allColumns.addAll(this.pkColumns); + allColumns.addAll(this.nonPkColumns); + } + + public String getTableName() { + return tableName; + } + + public List getPkColumns() { + return pkColumns; + } + + public List getNonPkColumns() { + return nonPkColumns; + } + + public List getAllColumns() { + return allColumns; + } + + @Override + public boolean equals(@javax.annotation.Nullable Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TrackedSpannerTable)) { + return false; + } + TrackedSpannerTable that = (TrackedSpannerTable) o; + return Objects.equals(tableName, that.tableName) + && Objects.equals(pkColumns, that.pkColumns) + && Objects.equals(nonPkColumns, that.nonPkColumns) + && Objects.equals(pkColumns, that.pkColumns); + } + + @Override + public String toString() { + return "TrackedSpannerTable{" + + "tableName='" + + tableName + + '\'' + + ", pkColumns=" + + pkColumns + + ", nonPkColumns=" + + nonPkColumns + + ", allColumns=" + + allColumns + + '}'; + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/package-info.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/package-info.java new file mode 100644 index 0000000000..6e3f757276 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/model/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 Google 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. + */ + +/** Google Cloud Teleport templates that process data within Google Cloud. */ +package com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model; diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/package-info.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/package-info.java new file mode 100644 index 0000000000..b21cd83212 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 Google 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. + */ + +/** Google Cloud Teleport templates that process data within Google Cloud. */ +package com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery; diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/BigQueryUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/BigQueryUtils.java new file mode 100644 index 0000000000..271ddff231 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/BigQueryUtils.java @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery.schemautils; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.Timestamp; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.Mod; +import java.util.HashSet; +import java.util.Set; + +/** {@link BigQueryUtils} providdes utils for processing BigQuery schema. */ +public class BigQueryUtils { + + public static final String BQ_CHANGELOG_FIELD_NAME_ORIGINAL_PAYLOAD_JSON = + "_metadata_spanner_original_payload_json"; + // TODO(haikuo-google): Create static variables for "_metadata_error" and "_metadata_retry_count" + // in com.google.cloud.teleport.v2.cdc.dlq.FileBasedDeadLetterQueueReconsumer and use them here. + public static final String BQ_CHANGELOG_FIELD_NAME_ERROR = "_metadata_error"; + public static final String BQ_CHANGELOG_FIELD_NAME_RETRY_COUNT = "_metadata_retry_count"; + public static final String BQ_CHANGELOG_FIELD_NAME_MOD_TYPE = "_metadata_spanner_mod_type"; + public static final String BQ_CHANGELOG_FIELD_NAME_TABLE_NAME = "_metadata_spanner_table_name"; + public static final String BQ_CHANGELOG_FIELD_NAME_SPANNER_COMMIT_TIMESTAMP = + "_metadata_spanner_commit_timestamp"; + public static final String BQ_CHANGELOG_FIELD_NAME_SERVER_TRANSACTION_ID = + "_metadata_spanner_server_transaction_id"; + public static final String BQ_CHANGELOG_FIELD_NAME_RECORD_SEQUENCE = + "_metadata_spanner_record_sequence"; + public static final String BQ_CHANGELOG_FIELD_NAME_IS_LAST_RECORD_IN_TRANSACTION_IN_PARTITION = + "_metadata_spanner_is_last_record_in_transaction_in_partition"; + public static final String BQ_CHANGELOG_FIELD_NAME_NUMBER_OF_RECORDS_IN_TRANSACTION = + "_metadata_spanner_number_of_records_in_transaction"; + public static final String BQ_CHANGELOG_FIELD_NAME_NUMBER_OF_PARTITIONS_IN_TRANSACTION = + "_metadata_spanner_number_of_partitions_in_transaction"; + public static final String BQ_CHANGELOG_FIELD_NAME_BIGQUERY_COMMIT_TIMESTAMP = + "_metadata_big_query_commit_timestamp"; + + /** + * @return the fields that are only used intermediately in the pipeline and are not corresponding + * to Spanner columns. + */ + public static Set getBigQueryIntermediateMetadataFieldNames() { + Set fieldNames = new HashSet<>(); + fieldNames.add(BQ_CHANGELOG_FIELD_NAME_ERROR); + fieldNames.add(BQ_CHANGELOG_FIELD_NAME_RETRY_COUNT); + fieldNames.add(BQ_CHANGELOG_FIELD_NAME_ORIGINAL_PAYLOAD_JSON); + return fieldNames; + } + + public static void setMetadataFiledsOfTableRow( + String spannerTableName, + Mod mod, + String modJsonString, + Timestamp spannerCommitTimestamp, + TableRow tableRow) { + tableRow.set(BQ_CHANGELOG_FIELD_NAME_ORIGINAL_PAYLOAD_JSON, modJsonString); + tableRow.set(BQ_CHANGELOG_FIELD_NAME_MOD_TYPE, mod.getModType().name()); + tableRow.set(BQ_CHANGELOG_FIELD_NAME_TABLE_NAME, spannerTableName); + tableRow.set( + BQ_CHANGELOG_FIELD_NAME_SPANNER_COMMIT_TIMESTAMP, spannerCommitTimestamp.toString()); + tableRow.set(BQ_CHANGELOG_FIELD_NAME_SERVER_TRANSACTION_ID, mod.getServerTransactionId()); + tableRow.set(BQ_CHANGELOG_FIELD_NAME_RECORD_SEQUENCE, mod.getRecordSequence()); + tableRow.set( + BQ_CHANGELOG_FIELD_NAME_IS_LAST_RECORD_IN_TRANSACTION_IN_PARTITION, + mod.getIsLastRecordInTransactionInPartition()); + tableRow.set( + BQ_CHANGELOG_FIELD_NAME_NUMBER_OF_RECORDS_IN_TRANSACTION, + mod.getNumberOfRecordsInTransaction()); + tableRow.set( + BQ_CHANGELOG_FIELD_NAME_NUMBER_OF_PARTITIONS_IN_TRANSACTION, + mod.getNumberOfPartitionsInTransaction()); + tableRow.set(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_BIGQUERY_COMMIT_TIMESTAMP, "AUTO"); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java new file mode 100644 index 0000000000..e3b1cc400a --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java @@ -0,0 +1,235 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery.schemautils; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.Type; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerColumn; +import java.util.ArrayList; +import java.util.List; +import org.json.JSONArray; +import org.json.JSONObject; + +/** + * The {@link SpannerToBigQueryUtils} provides methods that convert Spanner types to BigQuery types. + */ +public class SpannerToBigQueryUtils { + + public static List spannerColumnsToBigQueryIOFields( + List spannerColumns) { + final List bigQueryFields = new ArrayList<>(spannerColumns.size()); + for (TrackedSpannerColumn spannerColumn : spannerColumns) { + bigQueryFields.add(spannerColumnToBigQueryIOField(spannerColumn)); + } + + return bigQueryFields; + } + + private static TableFieldSchema spannerColumnToBigQueryIOField( + TrackedSpannerColumn spannerColumn) { + TableFieldSchema bigQueryField = + new TableFieldSchema().setName(spannerColumn.getName()).setMode(Field.Mode.REPEATED.name()); + Type spannerType = spannerColumn.getType(); + + if (spannerType.equals(Type.array(Type.bool()))) { + bigQueryField.setType("BOOL"); + } else if (spannerType.equals(Type.array(Type.bytes()))) { + bigQueryField.setType("BYTES"); + } else if (spannerType.equals(Type.array(Type.date()))) { + bigQueryField.setType("DATE"); + } else if (spannerType.equals(Type.array(Type.float64()))) { + bigQueryField.setType("FLOAT64"); + } else if (spannerType.equals(Type.array(Type.int64()))) { + bigQueryField.setType("INT64"); + } else if (spannerType.equals(Type.array(Type.json()))) { + bigQueryField.setType("STRING"); + } else if (spannerType.equals(Type.array(Type.numeric()))) { + bigQueryField.setType("NUMERIC"); + } else if (spannerType.equals(Type.array(Type.string()))) { + bigQueryField.setType("STRING"); + } else if (spannerType.equals(Type.array(Type.timestamp()))) { + bigQueryField.setType("TIMESTAMP"); + } else { + // Set NULLABLE for all non-array types, since we only insert primary key columns for deleted + // rows, which leaves non-primary key columns always null. + // E.g. if in Spanner schema we set "FirstName" which is non-primary key to NOT NULL, and we + // set the same field to NOT NULL in BigQuery, when we delete the Spanner row, we will not + // populate "FirstName" field in BigQuery, which violates the constraints. + bigQueryField.setMode(Field.Mode.NULLABLE.name()); + StandardSQLTypeName bigQueryType; + switch (spannerType.getCode()) { + case BOOL: + bigQueryType = StandardSQLTypeName.BOOL; + break; + case BYTES: + bigQueryType = StandardSQLTypeName.BYTES; + break; + case DATE: + bigQueryType = StandardSQLTypeName.DATE; + break; + case FLOAT64: + bigQueryType = StandardSQLTypeName.FLOAT64; + break; + case INT64: + bigQueryType = StandardSQLTypeName.INT64; + break; + case JSON: + bigQueryType = StandardSQLTypeName.STRING; + break; + case NUMERIC: + bigQueryType = StandardSQLTypeName.NUMERIC; + break; + case STRING: + bigQueryType = StandardSQLTypeName.STRING; + break; + case TIMESTAMP: + bigQueryType = StandardSQLTypeName.TIMESTAMP; + break; + default: + throw new IllegalArgumentException( + String.format("Unsupported Spanner type: %s", spannerType)); + } + bigQueryField.setType(bigQueryType.name()); + } + + return bigQueryField; + } + + public static void spannerSnapshotRowToBigQueryTableRow( + ResultSet resultSet, List spannerNonPkColumns, TableRow tableRow) { + if (resultSet.next()) { + for (TrackedSpannerColumn spannerNonPkColumn : spannerNonPkColumns) { + tableRow.set( + spannerNonPkColumn.getName(), + getColumnValueFromResultSet(spannerNonPkColumn, resultSet)); + } + } else { + throw new IllegalArgumentException( + "Received zero row from the result set of Spanner snapshot row"); + } + + if (resultSet.next()) { + throw new IllegalArgumentException( + "Received more than one rows from the result set of Spanner snapshot row"); + } + } + + private static Object getColumnValueFromResultSet( + TrackedSpannerColumn spannerColumn, ResultSet resultSet) { + String columnName = spannerColumn.getName(); + Type columnType = spannerColumn.getType(); + + if (resultSet.isNull(columnName)) { + return null; + } + + if (columnType.equals(Type.array(Type.bool()))) { + return resultSet.getBooleanList(columnName); + } else if (columnType.equals(Type.array(Type.bytes()))) { + List bytesList = resultSet.getBytesList(columnName); + List result = new ArrayList<>(); + for (ByteArray bytes : bytesList) { + result.add(bytes.toBase64()); + } + return result; + } else if (columnType.equals(Type.array(Type.date()))) { + List result = new ArrayList<>(); + for (Date date : resultSet.getDateList(columnName)) { + result.add(date.toString()); + } + return result; + } else if (columnType.equals(Type.array(Type.float64()))) { + return resultSet.getDoubleList(columnName); + } else if (columnType.equals(Type.array(Type.int64()))) { + return resultSet.getLongList(columnName); + } else if (columnType.equals(Type.array(Type.json()))) { + return resultSet.getJsonList(columnName); + } else if (columnType.equals(Type.array(Type.numeric()))) { + return resultSet.getBigDecimalList(columnName); + } else if (columnType.equals(Type.array(Type.string()))) { + return resultSet.getStringList(columnName); + } else if (columnType.equals(Type.array(Type.timestamp()))) { + List result = new ArrayList<>(); + for (Timestamp timestamp : resultSet.getTimestampList(columnName)) { + result.add(timestamp.toString()); + } + return result; + } else { + Type.Code columnTypeCode = columnType.getCode(); + switch (columnTypeCode) { + case BOOL: + return resultSet.getBoolean(columnName); + case BYTES: + return resultSet.getBytes(columnName).toBase64(); + case DATE: + return resultSet.getDate(columnName).toString(); + case FLOAT64: + return resultSet.getDouble(columnName); + case INT64: + return resultSet.getLong(columnName); + case JSON: + return resultSet.getJson(columnName); + case NUMERIC: + return resultSet.getBigDecimal(columnName); + case STRING: + return resultSet.getString(columnName); + case TIMESTAMP: + return resultSet.getTimestamp(columnName).toString(); + default: + throw new IllegalArgumentException( + String.format("Unsupported Spanner type: %s", columnTypeCode)); + } + } + } + + public static void addSpannerNonPkColumnsToTableRow( + String newValuesJson, List spannerNonPkColumns, TableRow tableRow) { + JSONObject newValuesJsonObject = new JSONObject(newValuesJson); + for (TrackedSpannerColumn spannerColumn : spannerNonPkColumns) { + String columnName = spannerColumn.getName(); + Type columnType = spannerColumn.getType(); + if (!newValuesJsonObject.has(columnName) || newValuesJsonObject.isNull(columnName)) { + continue; + } + + if (columnType.equals(Type.array(Type.bool())) + || columnType.equals(Type.array(Type.bytes())) + || columnType.equals(Type.array(Type.date())) + || columnType.equals(Type.array(Type.float64())) + || columnType.equals(Type.array(Type.int64())) + || columnType.equals(Type.array(Type.json())) + || columnType.equals(Type.array(Type.numeric())) + || columnType.equals(Type.array(Type.string())) + || columnType.equals(Type.array(Type.timestamp()))) { + JSONArray jsonArray = newValuesJsonObject.getJSONArray(columnName); + List objects = new ArrayList<>(jsonArray.length()); + for (Object o : jsonArray) { + objects.add(o); + } + tableRow.set(columnName, objects); + } else { + tableRow.set(columnName, newValuesJsonObject.get(columnName)); + } + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java new file mode 100644 index 0000000000..92e9d03a88 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java @@ -0,0 +1,360 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery.schemautils; + +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Key; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.Statement; +import com.google.cloud.spanner.Type; +import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerColumn; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerTable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.json.JSONObject; + +/** + * Class {@link SpannerUtils} provides methods that retrieve schema information from Spanner. Note + * all the models returned in the methods of these class are tracked by the change stream. + */ +public class SpannerUtils { + + private static final String INFORMATION_SCHEMA_TABLE_NAME = "TABLE_NAME"; + private static final String INFORMATION_SCHEMA_COLUMN_NAME = "COLUMN_NAME"; + private static final String INFORMATION_SCHEMA_SPANNER_TYPE = "SPANNER_TYPE"; + private static final String INFORMATION_SCHEMA_ORDINAL_POSITION = "ORDINAL_POSITION"; + private static final String INFORMATION_SCHEMA_CONSTRAINT_NAME = "CONSTRAINT_NAME"; + private static final String INFORMATION_SCHEMA_ALL = "ALL"; + + private DatabaseClient databaseClient; + private String changeStreamName; + + public SpannerUtils(DatabaseClient databaseClient, String changeStreamName) { + this.databaseClient = databaseClient; + this.changeStreamName = changeStreamName; + } + + /** + * @return a map where the key is the table name tracked by the change stream and the value is the + * {@link SpannerTable} object of the table name. This function should be called once in the + * initialization of the DoFn. + */ + public Map getSpannerTableByName() { + Set spannerTableNames = getSpannerTableNamesTrackedByChangeStreams(); + + Map> spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName = + getSpannerColumnNamesExplicitlyTrackedByChangeStreamsByTableName(); + + return getSpannerTableByName( + spannerTableNames, spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName); + } + + /** + * @return a map where the key is the table name tracked by the change stream and the value is the + * {@link SpannerTable} object of the table name. + */ + private Map getSpannerTableByName( + Set spannerTableNames, + Map> spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName) { + Map> spannerColumnsByTableName = + getSpannerColumnsByTableName( + spannerTableNames, spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName); + Map> keyColumnNameByTableName = + getKeyColumnNameByTableName(spannerTableNames); + + Map result = new HashMap<>(); + for (String tableName : spannerColumnsByTableName.keySet()) { + List pkColumns = new ArrayList<>(); + List nonPkColumns = new ArrayList<>(); + Set keyColumnNames = keyColumnNameByTableName.get(tableName); + for (TrackedSpannerColumn spannerColumn : spannerColumnsByTableName.get(tableName)) { + if (keyColumnNames.contains(spannerColumn.getName())) { + pkColumns.add(spannerColumn); + } else { + nonPkColumns.add(spannerColumn); + } + } + result.put(tableName, new TrackedSpannerTable(tableName, pkColumns, nonPkColumns)); + } + + return result; + } + + /** + * Query INFORMATION_SCHEMA.COLUMNS to construct {@link SpannerColumn} for each Spanner column + * tracked by change stream. + */ + private Map> getSpannerColumnsByTableName( + Set spannerTableNames, + Map> spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName) { + Map> result = new HashMap<>(); + StringBuilder sqlStringBuilder = + new StringBuilder( + "SELECT TABLE_NAME, COLUMN_NAME, ORDINAL_POSITION, SPANNER_TYPE " + + "FROM INFORMATION_SCHEMA.COLUMNS"); + + // Skip the columns of the tables that are not tracked by change stream. + if (!spannerTableNames.isEmpty()) { + sqlStringBuilder.append(" WHERE TABLE_NAME IN UNNEST (@tableNames)"); + } + + Statement.Builder statementBuilder = Statement.newBuilder(sqlStringBuilder.toString()); + if (!spannerTableNames.isEmpty()) { + statementBuilder.bind("tableNames").to(Value.stringArray(new ArrayList<>(spannerTableNames))); + } + + try (ResultSet columnsResultSet = + databaseClient.singleUse().executeQuery(statementBuilder.build())) { + while (columnsResultSet.next()) { + String tableName = columnsResultSet.getString(INFORMATION_SCHEMA_TABLE_NAME); + String columnName = columnsResultSet.getString(INFORMATION_SCHEMA_COLUMN_NAME); + // Skip if the columns of the table is tracked explicitly, and the specified column is not + // tracked. + if (spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName.containsKey(tableName) + && !spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName + .get(tableName) + .contains(columnName)) { + continue; + } + + int ordinalPosition = (int) columnsResultSet.getLong(INFORMATION_SCHEMA_ORDINAL_POSITION); + String spannerType = columnsResultSet.getString(INFORMATION_SCHEMA_SPANNER_TYPE); + result.putIfAbsent(tableName, new ArrayList<>()); + TrackedSpannerColumn spannerColumn = + TrackedSpannerColumn.create( + columnName, informationSchemaTypeToSpannerType(spannerType), ordinalPosition); + result.get(tableName).add(spannerColumn); + } + } + + return result; + } + + /** + * Query INFORMATION_SCHEMA.KEY_COLUMN_USAGE to get the names of the primary key columns that are + * tracked by change stream. We need to know the primary keys information to be able to set {@link + * TableRow} or do Spanner snapshot read, the alternative way is to extract the primary key + * information from {@link Mod} whenever we process it, but it's less efficient, since that will + * require to parse the types and sort them based on the ordinal positions for each {@link Mod}. + */ + private Map> getKeyColumnNameByTableName(Set spannerTableNames) { + Map> result = new HashMap<>(); + StringBuilder sqlStringBuilder = + new StringBuilder( + "SELECT TABLE_NAME, COLUMN_NAME, CONSTRAINT_NAME FROM" + + " INFORMATION_SCHEMA.KEY_COLUMN_USAGE"); + + // Skip the tables that are not tracked by change stream. + if (!spannerTableNames.isEmpty()) { + sqlStringBuilder.append(" WHERE TABLE_NAME IN UNNEST (@tableNames)"); + } + + Statement.Builder statementBuilder = Statement.newBuilder(sqlStringBuilder.toString()); + if (!spannerTableNames.isEmpty()) { + statementBuilder.bind("tableNames").to(Value.stringArray(new ArrayList<>(spannerTableNames))); + } + + try (ResultSet keyColumnsResultSet = + databaseClient.singleUse().executeQuery(statementBuilder.build())) { + while (keyColumnsResultSet.next()) { + String tableName = keyColumnsResultSet.getString(INFORMATION_SCHEMA_TABLE_NAME); + String columnName = keyColumnsResultSet.getString(INFORMATION_SCHEMA_COLUMN_NAME); + String constraintName = keyColumnsResultSet.getString(INFORMATION_SCHEMA_CONSTRAINT_NAME); + // We are only interested in primary key constraint. + if (isPrimaryKey(constraintName)) { + result.putIfAbsent(tableName, new HashSet<>()); + result.get(tableName).add(columnName); + } + } + } + + return result; + } + + private static boolean isPrimaryKey(String constraintName) { + return constraintName.startsWith("PK"); + } + + /** @return the Spanner table names that are tracked by the change stream. */ + private Set getSpannerTableNamesTrackedByChangeStreams() { + boolean isChangeStreamForAll = isChangeStreamForAll(); + + String sql = + "SELECT TABLE_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_TABLES " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + Statement.Builder statementBuilder = + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName); + + if (isChangeStreamForAll) { + // If the change stream is tracking all tables, we have to look up the table names in + // INFORMATION_SCHEMA.TABLES. + sql = "SELECT TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = \"\""; + statementBuilder = Statement.newBuilder(sql); + } + + Set result = new HashSet<>(); + try (ResultSet resultSet = databaseClient.singleUse().executeQuery(statementBuilder.build())) { + + while (resultSet.next()) { + result.add(resultSet.getString(INFORMATION_SCHEMA_TABLE_NAME)); + } + } + + return result; + } + + /** @return if the change stream tracks all the tables in the database. */ + private boolean isChangeStreamForAll() { + String sql = + "SELECT CHANGE_STREAMS.ALL FROM INFORMATION_SCHEMA.CHANGE_STREAMS " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + Boolean result = null; + try (ResultSet resultSet = + databaseClient + .singleUse() + .executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) { + + while (resultSet.next()) { + result = resultSet.getBoolean(INFORMATION_SCHEMA_ALL); + } + } + + if (result == null) { + throw new IllegalArgumentException( + String.format("Cannot find change stream %s in INFORMATION_SCHEMA", changeStreamName)); + } + + return result; + } + + /** + * @return the Spanner column names that are tracked explicitly by change stream by table name. + * e.g. Given Singers table with SingerId, FirstName and LastName, an empty map will be + * returned if we have change stream "CREATE CHANGE STREAM AllStream FOR ALL" or "CREATE + * CHANGE STREAM AllStream FOR Singers", {"Singers" -> {"SingerId", "FirstName"}} will be + * returned if we have change stream "CREATE CHANGE STREAM SingerStream FOR Singers(SingerId, + * FirstName)" + */ + private Map> + getSpannerColumnNamesExplicitlyTrackedByChangeStreamsByTableName() { + String sql = + "SELECT TABLE_NAME, COLUMN_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_COLUMNS " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + + Map> result = new HashMap<>(); + try (ResultSet resultSet = + databaseClient + .singleUse() + .executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) { + + while (resultSet.next()) { + String tableName = resultSet.getString(INFORMATION_SCHEMA_TABLE_NAME); + String columnName = resultSet.getString(INFORMATION_SCHEMA_COLUMN_NAME); + result.putIfAbsent(tableName, new HashSet<>()); + result.get(tableName).add(columnName); + } + } + + return result; + } + + private Type informationSchemaTypeToSpannerType(String type) { + type = cleanInformationSchemaType(type); + switch (type) { + case "BOOL": + return Type.bool(); + case "BYTES": + return Type.bytes(); + case "DATE": + return Type.date(); + case "FLOAT64": + return Type.float64(); + case "INT64": + return Type.int64(); + case "JSON": + return Type.json(); + case "NUMERIC": + return Type.numeric(); + case "STRING": + return Type.string(); + case "TIMESTAMP": + return Type.timestamp(); + default: + if (type.startsWith("ARRAY")) { + // Get array type, e.g. "ARRAY" -> "STRING". + String spannerArrayType = type.substring(6, type.length() - 1); + Type itemType = informationSchemaTypeToSpannerType(spannerArrayType); + return Type.array(itemType); + } + + throw new IllegalArgumentException(String.format("Unsupported Spanner type: %s", type)); + } + } + + /** + * Remove the Spanner type length limit, since Spanner doesn't document clearly on the + * parameterized types like BigQuery does, i.e. BigQuery's docmentation on Parameterized + * data types, but Spanner doesn't have a similar one. We might have problem if we transfer + * the length limit into BigQuery. By removing the length limit, we essentially loose the + * constraint of data written to BigQuery, and it won't cause errors. + */ + private String cleanInformationSchemaType(String type) { + // Remove type size, e.g. STRING(1024) -> STRING. + int leftParenthesisIdx = type.indexOf('('); + if (leftParenthesisIdx != -1) { + type = type.substring(0, leftParenthesisIdx) + type.substring(type.indexOf(')') + 1); + } + + // Convert it to upper case. + return type.toUpperCase(); + } + + public static void appendToSpannerKey( + TrackedSpannerColumn column, JSONObject keysJsonObject, Key.Builder keyBuilder) { + Type.Code code = column.getType().getCode(); + String name = column.getName(); + switch (code) { + case BOOL: + keyBuilder.append(keysJsonObject.getBoolean(name)); + break; + case FLOAT64: + keyBuilder.append(keysJsonObject.getDouble(name)); + break; + case INT64: + keyBuilder.append(keysJsonObject.getLong(name)); + break; + case NUMERIC: + keyBuilder.append(keysJsonObject.getBigDecimal(name)); + break; + case BYTES: + case DATE: + case STRING: + case TIMESTAMP: + keyBuilder.append(keysJsonObject.getString(name)); + break; + default: + throw new IllegalArgumentException(String.format("Unsupported Spanner type: %s", code)); + } + } +} diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/package-info.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/package-info.java new file mode 100644 index 0000000000..ec683271e3 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 Google 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. + */ + +/** Google Cloud Teleport templates that process data within Google Cloud. */ +package com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils; diff --git a/v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-bigquery-command-spec.json b/v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-bigquery-command-spec.json new file mode 100644 index 0000000000..bb6e00f70c --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/resources/spanner-changestreams-to-bigquery-command-spec.json @@ -0,0 +1,7 @@ +{ + "mainClass": "com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.SpannerChangeStreamsToBigQuery", + "classPath": "/template/spanner-changestreams-to-bigquery/*:/template/spanner-changestreams-to-bigquery/libs/*:/template/spanner-changestreams-to-bigquery/classes", + "defaultParameterValues": { + "labels": "{\"goog-dataflow-provided-template-type\":\"flex\", \"goog-dataflow-provided-template-name\":\"spanner-changestreams-to-bigquery\"}" + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java new file mode 100644 index 0000000000..d50ab63354 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java @@ -0,0 +1,478 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import static com.google.common.truth.Truth.assertThat; +import static org.mockito.Mockito.when; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Key; +import com.google.cloud.spanner.ReadContext; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.ResultSets; +import com.google.cloud.spanner.Statement; +import com.google.cloud.spanner.Struct; +import com.google.cloud.spanner.Type; +import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerColumn; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerTable; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; +import com.google.common.collect.ImmutableList; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.json.JSONObject; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +/** Test class for {@link SchemaUtilsTest}. */ +@RunWith(MockitoJUnitRunner.class) +public class SchemaUtilsTest { + + private static final String changeStreamName = "changeStreamName"; + @Mock private DatabaseClient mockDatabaseClient; + @Mock private ReadContext mockReadContext; + private List spannerColumnsOfAllTypes; + + @Before + public void setUp() { + when(mockDatabaseClient.singleUse()).thenReturn(mockReadContext); + spannerColumnsOfAllTypes = + ImmutableList.of( + TrackedSpannerColumn.create("BoolCol", Type.bool(), 1), + TrackedSpannerColumn.create("BytesCol", Type.bytes(), 2), + TrackedSpannerColumn.create("DateCol", Type.date(), 3), + TrackedSpannerColumn.create("Float64Col", Type.float64(), 4), + TrackedSpannerColumn.create("Int64Col", Type.int64(), 5), + TrackedSpannerColumn.create("JsonCol", Type.json(), 6), + TrackedSpannerColumn.create("NumericCol", Type.numeric(), 7), + TrackedSpannerColumn.create("StringCol", Type.string(), 8), + TrackedSpannerColumn.create("TimestampCol", Type.timestamp(), 9), + TrackedSpannerColumn.create("BoolArrayCol", Type.array(Type.bool()), 10), + TrackedSpannerColumn.create("BytesArrayCol", Type.array(Type.bytes()), 11), + TrackedSpannerColumn.create("DateArrayCol", Type.array(Type.date()), 12), + TrackedSpannerColumn.create("Float64ArrayCol", Type.array(Type.float64()), 13), + TrackedSpannerColumn.create("Int64ArrayCol", Type.array(Type.int64()), 14), + TrackedSpannerColumn.create("JsonArrayCol", Type.array(Type.json()), 15), + TrackedSpannerColumn.create("NumericArrayCol", Type.array(Type.numeric()), 16), + TrackedSpannerColumn.create("StringArrayCol", Type.array(Type.string()), 17), + TrackedSpannerColumn.create("TimestampArrayCol", Type.array(Type.timestamp()), 18)); + } + + @Test + public void testChangeStreamTrackAll() { + mockInformationSchemaChangeStreamsQuery(true); + mockInformationSchemaTablesQuery(); + mockInformationSchemaColumnsQuery(); + mockInformationSchemaKeyColumnUsageQuery(); + String sql = + "SELECT TABLE_NAME, COLUMN_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_COLUMNS " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + when(mockReadContext.executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) + .thenReturn( + ResultSets.forRows( + Type.struct( + Type.StructField.of("TABLE_NAME", Type.string()), + Type.StructField.of("COLUMN_NAME", Type.string())), + Collections.emptyList())); + + Map actualSpannerTableByName = + new SpannerUtils(mockDatabaseClient, changeStreamName).getSpannerTableByName(); + + List singersPkColumns = + ImmutableList.of(TrackedSpannerColumn.create("SingerId", Type.int64(), 1)); + List singersNonPkColumns = + ImmutableList.of( + TrackedSpannerColumn.create("FirstName", Type.string(), 2), + TrackedSpannerColumn.create("LastName", Type.string(), 3)); + Map expectedSpannerTableByName = new HashMap<>(); + expectedSpannerTableByName.put( + "Singers", new TrackedSpannerTable("Singers", singersPkColumns, singersNonPkColumns)); + assertThat(actualSpannerTableByName).isEqualTo(expectedSpannerTableByName); + } + + @Test + public void testChangeStreamTrackOneTable() { + mockInformationSchemaChangeStreamsQuery(false); + mockInformationSchemaChangeStreamTablesQuery(); + mockInformationSchemaColumnsQuery(); + mockInformationSchemaKeyColumnUsageQuery(); + String sql = + "SELECT TABLE_NAME, COLUMN_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_COLUMNS " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + when(mockReadContext.executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) + .thenReturn( + ResultSets.forRows( + Type.struct( + Type.StructField.of("TABLE_NAME", Type.string()), + Type.StructField.of("COLUMN_NAME", Type.string())), + Collections.emptyList())); + + Map actualSpannerTableByName = + new SpannerUtils(mockDatabaseClient, changeStreamName).getSpannerTableByName(); + + List singersPkColumns = + ImmutableList.of(TrackedSpannerColumn.create("SingerId", Type.int64(), 1)); + List singersNonPkColumns = + ImmutableList.of( + TrackedSpannerColumn.create("FirstName", Type.string(), 2), + TrackedSpannerColumn.create("LastName", Type.string(), 3)); + Map expectedSpannerTableByName = new HashMap<>(); + expectedSpannerTableByName.put( + "Singers", new TrackedSpannerTable("Singers", singersPkColumns, singersNonPkColumns)); + assertThat(actualSpannerTableByName).isEqualTo(expectedSpannerTableByName); + } + + @Test + public void testChangeStreamTrackTwoColumns() { + mockInformationSchemaChangeStreamsQuery(false); + mockInformationSchemaChangeStreamTablesQuery(); + mockInformationSchemaColumnsQuery(); + mockInformationSchemaKeyColumnUsageQuery(); + String sql = + "SELECT TABLE_NAME, COLUMN_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_COLUMNS " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + when(mockReadContext.executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) + .thenReturn( + ResultSets.forRows( + Type.struct( + Type.StructField.of("TABLE_NAME", Type.string()), + Type.StructField.of("COLUMN_NAME", Type.string())), + ImmutableList.of( + Struct.newBuilder() + .set("TABLE_NAME") + .to(Value.string("Singers")) + .set("COLUMN_NAME") + .to(Value.string("SingerId")) + .build(), + Struct.newBuilder() + .set("TABLE_NAME") + .to(Value.string("Singers")) + .set("COLUMN_NAME") + .to(Value.string("FirstName")) + .build()))); + + Map actualSpannerTableByName = + new SpannerUtils(mockDatabaseClient, changeStreamName).getSpannerTableByName(); + + List singersPkColumns = + Collections.singletonList(TrackedSpannerColumn.create("SingerId", Type.int64(), 1)); + List singersNonPkColumns = + Collections.singletonList(TrackedSpannerColumn.create("FirstName", Type.string(), 2)); + Map expectedSpannerTableByName = new HashMap<>(); + expectedSpannerTableByName.put( + "Singers", new TrackedSpannerTable("Singers", singersPkColumns, singersNonPkColumns)); + assertThat(actualSpannerTableByName).isEqualTo(expectedSpannerTableByName); + } + + @Test + public void testSpannerSnapshotRowToBigQueryTableRow() { + TableRow tableRow = new TableRow(); + Boolean[] booleanArray = {true, false, true}; + ByteArray[] bytesArray = { + ByteArray.copyFrom("123"), ByteArray.copyFrom("456"), ByteArray.copyFrom("789") + }; + Date[] dateArray = {Date.fromYearMonthDay(2022, 1, 22), Date.fromYearMonthDay(2022, 3, 11)}; + double[] float64Array = {Double.MIN_VALUE, Double.MAX_VALUE, 0, 1, -1, 1.2341}; + long[] int64Array = {Long.MAX_VALUE, Long.MIN_VALUE, 0, 1, -1}; + String[] jsonArray = {"{}", "{\"color\":\"red\",\"value\":\"#f00\"}", "[]"}; + BigDecimal[] numericArray = { + BigDecimal.valueOf(1, Integer.MAX_VALUE), + BigDecimal.valueOf(1, Integer.MIN_VALUE), + BigDecimal.ZERO, + BigDecimal.TEN, + BigDecimal.valueOf(3141592, 6) + }; + String[] stringArray = {"abc", "def", "ghi"}; + Timestamp[] timestampArray = { + Timestamp.ofTimeSecondsAndNanos(1646617853L, 972000000), + Timestamp.ofTimeSecondsAndNanos(1646637853L, 572000000), + Timestamp.ofTimeSecondsAndNanos(1646657853L, 772000000) + }; + List structFields = new ArrayList<>(spannerColumnsOfAllTypes.size()); + for (TrackedSpannerColumn spannerColumn : spannerColumnsOfAllTypes) { + structFields.add(Type.StructField.of(spannerColumn.getName(), spannerColumn.getType())); + } + // spotless:off + ResultSet resultSet = + ResultSets.forRows( + Type.struct(structFields), + Collections.singletonList( + Struct.newBuilder() + .set("BoolCol").to(Value.bool(true)) + .set("BytesCol").to(Value.bytes(ByteArray.copyFrom("123"))) + .set("DateCol").to(Date.fromYearMonthDay(2022, 1, 22)) + .set("Float64Col").to(Value.float64(1.2)) + .set("Int64Col").to(Value.int64(20)) + .set("JsonCol").to(Value.json("{\"color\":\"red\",\"value\":\"#f00\"}")) + .set("NumericCol").to(Value.numeric(BigDecimal.valueOf(123, 2))) + .set("StringCol").to(Value.string("abc")) + .set("TimestampCol").to(Timestamp.ofTimeSecondsAndNanos(1646617853L, 972000000)) + .set("BoolArrayCol").to(Value.boolArray(Arrays.asList(booleanArray))) + .set("BytesArrayCol").to(Value.bytesArray(Arrays.asList(bytesArray))) + .set("DateArrayCol").to(Value.dateArray(Arrays.asList(dateArray))) + .set("Float64ArrayCol").to(Value.float64Array(float64Array)) + .set("Int64ArrayCol").to(Value.int64Array(int64Array)) + .set("JsonArrayCol").to(Value.jsonArray(Arrays.asList(jsonArray))) + .set("NumericArrayCol").to(Value.numericArray(Arrays.asList(numericArray))) + .set("StringArrayCol").to(Value.stringArray(Arrays.asList(stringArray))) + .set("TimestampArrayCol") + .to(Value.timestampArray(Arrays.asList(timestampArray))) + .build())); + // spotless:on + SpannerToBigQueryUtils.spannerSnapshotRowToBigQueryTableRow( + resultSet, spannerColumnsOfAllTypes, tableRow); + + assertThat(tableRow.toString()) + .isEqualTo( + "GenericData{classInfo=[f], {BoolCol=true, BytesCol=MTIz, DateCol=2022-01-22," + + " Float64Col=1.2, Int64Col=20, JsonCol={\"color\":\"red\",\"value\":\"#f00\"}," + + " NumericCol=1.23, StringCol=abc, TimestampCol=2022-03-07T01:50:53.972000000Z," + + " BoolArrayCol=[true, false, true], BytesArrayCol=[MTIz, NDU2, Nzg5]," + + " DateArrayCol=[2022-01-22, 2022-03-11], Float64ArrayCol=[4.9E-324," + + " 1.7976931348623157E308, 0.0, 1.0, -1.0, 1.2341]," + + " Int64ArrayCol=[9223372036854775807, -9223372036854775808, 0, 1, -1]," + + " JsonArrayCol=[{}, {\"color\":\"red\",\"value\":\"#f00\"}, []]," + + " NumericArrayCol=[1E-2147483647, 1E+2147483648, 0, 10, 3.141592]," + + " StringArrayCol=[abc, def, ghi]," + + " TimestampArrayCol=[2022-03-07T01:50:53.972000000Z," + + " 2022-03-07T07:24:13.572000000Z, 2022-03-07T12:57:33.772000000Z]}}"); + } + + @Test + public void testAppendToSpannerKey() { + JSONObject keysJsonObject = new JSONObject(); + keysJsonObject.put("BoolCol", true); + keysJsonObject.put("BytesCol", ByteArray.copyFrom("123").toBase64()); + keysJsonObject.put("DateCol", "2022-01-22"); + keysJsonObject.put("Float64Col", 1.2); + keysJsonObject.put("Int64Col", 20); + keysJsonObject.put("NumericCol", new BigDecimal(3.141592)); + keysJsonObject.put("StringCol", "abc"); + keysJsonObject.put("TimestampCol", "2022-03-07T01:50:53.972000000Z"); + Key.Builder keyBuilder = com.google.cloud.spanner.Key.newBuilder(); + for (TrackedSpannerColumn spannerColumn : spannerColumnsOfAllTypes) { + String typeName = spannerColumn.getType().getCode().name(); + // Array and JSON are not valid Spanner key type. + if (typeName.equals("ARRAY") || typeName.equals("JSON")) { + continue; + } + SpannerUtils.appendToSpannerKey(spannerColumn, keysJsonObject, keyBuilder); + } + + assertThat(keyBuilder.build().toString()) + .isEqualTo( + "[true,MTIz,2022-01-22,1.2,20," + + "3.14159200000000016217427400988526642322540283203125,abc," + + "2022-03-07T01:50:53.972000000Z]"); + } + + @Test + public void testSpannerColumnsToBigQueryIOFields() { + String bigQueryIOFieldsStr = + SpannerToBigQueryUtils.spannerColumnsToBigQueryIOFields(spannerColumnsOfAllTypes) + .toString(); + // Remove redundant information. + bigQueryIOFieldsStr = + bigQueryIOFieldsStr.replace( + "classInfo=[categories, collationSpec, description, fields, maxLength, mode, name," + + " policyTags, precision, scale, type], ", + ""); + bigQueryIOFieldsStr = bigQueryIOFieldsStr.replace("GenericData", ""); + + assertThat(bigQueryIOFieldsStr) + .isEqualTo( + "[{{mode=NULLABLE, name=BoolCol, type=BOOL}}, " + + "{{mode=NULLABLE, name=BytesCol, type=BYTES}}, " + + "{{mode=NULLABLE, name=DateCol, type=DATE}}, " + + "{{mode=NULLABLE, name=Float64Col, type=FLOAT64}}, " + + "{{mode=NULLABLE, name=Int64Col, type=INT64}}, " + + "{{mode=NULLABLE, name=JsonCol, type=STRING}}, " + + "{{mode=NULLABLE, name=NumericCol, type=NUMERIC}}, " + + "{{mode=NULLABLE, name=StringCol, type=STRING}}, " + + "{{mode=NULLABLE, name=TimestampCol, type=TIMESTAMP}}, " + + "{{mode=REPEATED, name=BoolArrayCol, type=BOOL}}, " + + "{{mode=REPEATED, name=BytesArrayCol, type=BYTES}}, " + + "{{mode=REPEATED, name=DateArrayCol, type=DATE}}, " + + "{{mode=REPEATED, name=Float64ArrayCol, type=FLOAT64}}, " + + "{{mode=REPEATED, name=Int64ArrayCol, type=INT64}}, " + + "{{mode=REPEATED, name=JsonArrayCol, type=STRING}}, " + + "{{mode=REPEATED, name=NumericArrayCol, type=NUMERIC}}, " + + "{{mode=REPEATED, name=StringArrayCol, type=STRING}}, " + + "{{mode=REPEATED, name=TimestampArrayCol, type=TIMESTAMP}}]"); + } + + @Test + public void testAddSpannerNonPkColumnsToTableRow() { + String newValuesJson = + "{\"BoolCol\":true,\"BytesCol\":\"ZmZm\",\"DateCol\":\"2020-12-12\",\"Float64Col\":1.3," + + "\"Int64Col\":\"5\"," + + "\"JsonCol\":\"{\\\"color\\\":\\\"red\\\",\\\"value\\\":\\\"#f00\\\"}\"," + + "\"NumericCol\":\"4.4\",\"StringCol\":\"abc\"," + + "\"TimestampCol\":\"2022-03-19T18:51:33.963910279Z\",\"BoolArrayCol\":[true,false]," + + "\"BytesArrayCol\":[\"YWJj\",\"YmNk\"]," + + "\"DateArrayCol\":[\"2021-01-22\",\"2022-01-01\"],\"Float64ArrayCol\":[1.2,4.4]," + + "\"Int64ArrayCol\":[\"1\",\"2\"]," + + "\"JsonArrayCol\":[\"{}\",\"{\\\"color\\\":\\\"red\\\",\\\"value\\\":\\\"#f00\\\"}\"," + + "\"[]\"]," + + "\"NumericArrayCol\":[\"2.2\",\"3.3\"],\"StringArrayCol\":[\"a\",\"b\"]," + + "\"TimestampArrayCol\":[\"2022-03-19T18:51:33.963910279Z\"," + + "\"2022-03-19T18:51:33.963910279Z\"]}"; + TableRow tableRow = new TableRow(); + SpannerToBigQueryUtils.addSpannerNonPkColumnsToTableRow( + newValuesJson, spannerColumnsOfAllTypes, tableRow); + + assertThat(tableRow.toString()) + .isEqualTo( + "GenericData{classInfo=[f], " + + "{BoolCol=true, BytesCol=ZmZm, DateCol=2020-12-12, Float64Col=1.3, Int64Col=5, " + + "JsonCol={\"color\":\"red\",\"value\":\"#f00\"}, NumericCol=4.4, StringCol=abc, " + + "TimestampCol=2022-03-19T18:51:33.963910279Z, BoolArrayCol=[true, false], " + + "BytesArrayCol=[YWJj, YmNk], DateArrayCol=[2021-01-22, 2022-01-01], " + + "Float64ArrayCol=[1.2, 4.4], Int64ArrayCol=[1, 2], " + + "JsonArrayCol=[{}, {\"color\":\"red\",\"value\":\"#f00\"}, []], " + + "NumericArrayCol=[2.2, 3.3], StringArrayCol=[a, b], " + + "TimestampArrayCol=[2022-03-19T18:51:33.963910279Z, " + + "2022-03-19T18:51:33.963910279Z]}}"); + } + + private void mockInformationSchemaChangeStreamsQuery(boolean isTrackingAll) { + String sql = + "SELECT CHANGE_STREAMS.ALL FROM INFORMATION_SCHEMA.CHANGE_STREAMS " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + + when(mockReadContext.executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) + .thenReturn( + ResultSets.forRows( + Type.struct(Type.StructField.of("ALL", Type.bool())), + Collections.singletonList( + Struct.newBuilder().set("ALL").to(Value.bool(isTrackingAll)).build()))); + } + + private void mockInformationSchemaTablesQuery() { + String sql = "SELECT TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = \"\""; + + when(mockReadContext.executeQuery(Statement.of(sql))) + .thenReturn( + ResultSets.forRows( + Type.struct(Type.StructField.of("TABLE_NAME", Type.string())), + ImmutableList.of( + Struct.newBuilder().set("TABLE_NAME").to(Value.string("Singers")).build()))); + } + + private void mockInformationSchemaColumnsQuery() { + String sql = + "SELECT TABLE_NAME, COLUMN_NAME, ORDINAL_POSITION, SPANNER_TYPE " + + "FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME IN UNNEST (@tableNames)"; + List tableNames = new ArrayList<>(); + tableNames.add("Singers"); + // spotless:off + List rows = + new ArrayList<>( + ImmutableList.of( + Struct.newBuilder() + .set("TABLE_NAME").to(Value.string("Singers")) + .set("COLUMN_NAME").to(Value.string("SingerId")) + .set("ORDINAL_POSITION").to(Value.int64(1)) + .set("SPANNER_TYPE").to(Value.string("INT64")) + .build(), + Struct.newBuilder() + .set("TABLE_NAME").to(Value.string("Singers")) + .set("COLUMN_NAME").to(Value.string("FirstName")) + .set("ORDINAL_POSITION").to(Value.int64(2)) + .set("SPANNER_TYPE").to(Value.string("STRING(1024)")) + .build(), + Struct.newBuilder() + .set("TABLE_NAME").to(Value.string("Singers")) + .set("COLUMN_NAME").to(Value.string("LastName")) + .set("ORDINAL_POSITION").to(Value.int64(3)) + .set("SPANNER_TYPE").to(Value.string("STRING")) + .build())); + // spotless:on + + when(mockReadContext.executeQuery( + Statement.newBuilder(sql) + .bind("tableNames") + .to(Value.stringArray(new ArrayList<>(tableNames))) + .build())) + .thenReturn( + ResultSets.forRows( + Type.struct( + Type.StructField.of("TABLE_NAME", Type.string()), + Type.StructField.of("COLUMN_NAME", Type.string()), + Type.StructField.of("ORDINAL_POSITION", Type.int64()), + Type.StructField.of("SPANNER_TYPE", Type.string())), + rows)); + } + + private void mockInformationSchemaChangeStreamTablesQuery() { + String sql = + "SELECT TABLE_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_TABLES " + + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + + when(mockReadContext.executeQuery( + Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) + .thenReturn( + ResultSets.forRows( + Type.struct(Type.StructField.of("TABLE_NAME", Type.string())), + Collections.singletonList( + Struct.newBuilder().set("TABLE_NAME").to(Value.string("Singers")).build()))); + } + + private void mockInformationSchemaKeyColumnUsageQuery() { + String sql = + "SELECT TABLE_NAME, COLUMN_NAME, CONSTRAINT_NAME FROM INFORMATION_SCHEMA.KEY_COLUMN_USAGE " + + "WHERE TABLE_NAME IN UNNEST (@tableNames)"; + List tableNames = new ArrayList<>(); + tableNames.add("Singers"); + // spotless:off + List rows = + new ArrayList<>( + Collections.singletonList( + Struct.newBuilder() + .set("TABLE_NAME").to(Value.string("Singers")) + .set("COLUMN_NAME").to(Value.string("SingerId")) + .set("CONSTRAINT_NAME").to(Value.string("PK_Singers")) + .build())); + // spotless:on + + when(mockReadContext.executeQuery( + Statement.newBuilder(sql).bind("tableNames").to(Value.stringArray(tableNames)).build())) + .thenReturn( + ResultSets.forRows( + Type.struct( + Type.StructField.of("TABLE_NAME", Type.string()), + Type.StructField.of("COLUMN_NAME", Type.string()), + Type.StructField.of("CONSTRAINT_NAME", Type.string())), + rows)); + } +} From 491c0c28cccf412d31ebc4d4dc4c83475f003e82 Mon Sep 17 00:00:00 2001 From: dhercher Date: Fri, 22 Apr 2022 02:21:31 -0700 Subject: [PATCH 106/145] DatastreamToSQL: Use KV into the write PiperOrigin-RevId: 443607086 --- .../teleport/v2/templates/DataStreamToSQL.java | 13 +++++++------ .../cloud/teleport/v2/transforms/ProcessDml.java | 8 +++----- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java index f69ff9b486..325e32a53a 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSQL.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -289,7 +290,7 @@ public static PipelineResult run(Options options) { * a) Convert JSON String FailsafeElements to TableRow's (tableRowRecords) * Stage 3) Filter stale rows using stateful PK transform */ - PCollection dmlStatements = + PCollection> dmlStatements = datastreamJsonRecords .apply("Format to DML", CreateDml.of(dataSourceConfiguration).withSchemaMap(schemaMap)) .apply("DML Stateful Processing", ProcessDml.statefulOrderByPK()); @@ -299,13 +300,13 @@ public static PipelineResult run(Options options) { */ dmlStatements.apply( "Write to SQL", - CdcJdbcIO.write() + CdcJdbcIO.>write() .withDataSourceConfiguration(dataSourceConfiguration) .withStatementFormatter( - new CdcJdbcIO.StatementFormatter() { - public String formatStatement(DmlInfo element) { - LOG.debug("Executing SQL: {}", element.getDmlSql()); - return element.getDmlSql(); + new CdcJdbcIO.StatementFormatter>() { + public String formatStatement(KV element) { + LOG.debug("Executing SQL: {}", element.getValue().getDmlSql()); + return element.getValue().getDmlSql(); } })); diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/ProcessDml.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/ProcessDml.java index 14481254ff..d521f01f7f 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/ProcessDml.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/transforms/ProcessDml.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; -import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; @@ -54,19 +53,18 @@ public static StatefulProcessDml statefulOrderByPK() { /** This class is used as the default return value of {@link ProcessDml#statefulOrderByPK()}. */ public static class StatefulProcessDml - extends PTransform>, PCollection> { + extends PTransform>, PCollection>> { public StatefulProcessDml() {} @Override - public PCollection expand(PCollection> input) { + public PCollection> expand(PCollection> input) { return input .apply(ParDo.of(new StatefulProcessDmlFn())) .apply( "Creating " + WINDOW_DURATION + " Window", Window.into(FixedWindows.of(DurationUtils.parseDuration(WINDOW_DURATION)))) - .apply(Reshuffle.of()) - .apply(Values.create()); + .apply(Reshuffle.of()); } } From 292c0dfdc5d3b1d0e68014ff96bda6651e8347ed Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Fri, 22 Apr 2022 10:28:03 -0700 Subject: [PATCH 107/145] Add parameter to allow users to specify if HTTP messages sent to HEC endpoint should be GZIP encoded. PiperOrigin-RevId: 443700293 --- .../teleport/splunk/HttpEventPublisher.java | 26 +++++++++++++++ .../teleport/splunk/SplunkEventWriter.java | 31 +++++++++++++++++- .../cloud/teleport/splunk/SplunkIO.java | 32 ++++++++++++++++++- .../teleport/templates/PubSubToSplunk.java | 1 + .../templates/common/SplunkConverters.java | 7 ++++ .../splunk/HttpEventPublisherTest.java | 10 +++++- 6 files changed, 104 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java b/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java index 2bc002809c..cc0a6b1397 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java +++ b/src/main/java/com/google/cloud/teleport/splunk/HttpEventPublisher.java @@ -18,6 +18,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.api.client.http.ByteArrayContent; +import com.google.api.client.http.GZipEncoding; import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpBackOffIOExceptionHandler; import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler; @@ -111,6 +112,8 @@ public static Builder newBuilder() { abstract Boolean disableCertificateValidation(); + abstract Boolean enableGzipHttpCompression(); + /** * Executes a POST for the list of {@link SplunkEvent} objects into Splunk's Http Event Collector * endpoint. @@ -123,6 +126,10 @@ public HttpResponse execute(List events) throws IOException { HttpContent content = getContent(events); HttpRequest request = requestFactory().buildPostRequest(genericUrl(), content); + if (enableGzipHttpCompression()) { + request.setEncoding(new GZipEncoding()); + } + HttpBackOffUnsuccessfulResponseHandler responseHandler = new HttpBackOffUnsuccessfulResponseHandler(getConfiguredBackOff()); responseHandler.setBackOffRequired(BackOffRequired.ON_SERVER_ERROR); @@ -173,6 +180,10 @@ public void close() throws IOException { */ private void setHeaders(HttpRequest request, String token) { request.getHeaders().setAuthorization(String.format(AUTHORIZATION_SCHEME, token)); + + if (enableGzipHttpCompression()) { + request.getHeaders().setContentEncoding("gzip"); + } } /** @@ -218,6 +229,8 @@ abstract static class Builder { abstract Boolean disableCertificateValidation(); + abstract Builder setEnableGzipHttpCompression(Boolean enableGzipHttpCompression); + abstract Builder setRootCaCertificate(byte[] certificate); abstract byte[] rootCaCertificate(); @@ -275,6 +288,19 @@ public Builder withDisableCertificateValidation(Boolean disableCertificateValida return setDisableCertificateValidation(disableCertificateValidation); } + /** + * Method to specify if HTTP requests sent to Splunk HEC should be GZIP encoded. + * + * @param enableGzipHttpCompression whether to enable Gzip encoding. + * @return {@link Builder} + */ + public Builder withEnableGzipHttpCompression(Boolean enableGzipHttpCompression) { + checkNotNull( + enableGzipHttpCompression, + "withEnableGzipHttpCompression(enableGzipHttpCompression) called with null input."); + return setEnableGzipHttpCompression(enableGzipHttpCompression); + } + /** * Method to max timeout for {@link ExponentialBackOff}. Otherwise uses the default setting for * {@link ExponentialBackOff}. diff --git a/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java b/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java index 43a0e446bf..b98dbeece2 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java +++ b/src/main/java/com/google/cloud/teleport/splunk/SplunkEventWriter.java @@ -64,6 +64,7 @@ public abstract class SplunkEventWriter extends DoFn, S private static final Integer DEFAULT_BATCH_COUNT = 1; private static final Boolean DEFAULT_DISABLE_CERTIFICATE_VALIDATION = false; private static final Boolean DEFAULT_ENABLE_BATCH_LOGS = true; + private static final Boolean DEFAULT_ENABLE_GZIP_HTTP_COMPRESSION = true; private static final Logger LOG = LoggerFactory.getLogger(SplunkEventWriter.class); private static final long DEFAULT_FLUSH_DELAY = 2; private static final Counter INPUT_COUNTER = @@ -106,6 +107,7 @@ public abstract class SplunkEventWriter extends DoFn, S private Integer batchCount; private Boolean disableValidation; private Boolean enableBatchLogs; + private Boolean enableGzipHttpCompression; private HttpEventPublisher publisher; private static final Gson GSON = @@ -130,6 +132,9 @@ public static Builder newBuilder() { @Nullable abstract ValueProvider enableBatchLogs(); + @Nullable + abstract ValueProvider enableGzipHttpCompression(); + @Nullable abstract ValueProvider inputBatchCount(); @@ -161,6 +166,17 @@ public void setup() { LOG.info("Enable Batch logs set to: {}", enableBatchLogs); } + if (enableGzipHttpCompression == null) { + + if (enableGzipHttpCompression() != null) { + enableGzipHttpCompression = enableGzipHttpCompression().get(); + } + + enableGzipHttpCompression = + MoreObjects.firstNonNull(enableGzipHttpCompression, DEFAULT_ENABLE_GZIP_HTTP_COMPRESSION); + LOG.info("Enable gzip http compression set to: {}", enableGzipHttpCompression); + } + // Either user supplied or default disableValidation. if (disableValidation == null) { @@ -178,7 +194,8 @@ public void setup() { HttpEventPublisher.newBuilder() .withUrl(url().get()) .withToken(token().get()) - .withDisableCertificateValidation(disableValidation); + .withDisableCertificateValidation(disableValidation) + .withEnableGzipHttpCompression(enableGzipHttpCompression); if (rootCaCertificatePath() != null && rootCaCertificatePath().get() != null) { builder.withRootCaCertificate(GCSUtils.getGcsFileAsBytes(rootCaCertificatePath().get())); @@ -441,6 +458,8 @@ abstract Builder setDisableCertificateValidation( abstract Builder setEnableBatchLogs(ValueProvider enableBatchLogs); + abstract Builder setEnableGzipHttpCompression(ValueProvider enableGzipHttpCompression); + abstract Builder setInputBatchCount(ValueProvider inputBatchCount); abstract SplunkEventWriter autoBuild(); @@ -534,6 +553,16 @@ public Builder withEnableBatchLogs(ValueProvider enableBatchLogs) { return setEnableBatchLogs(enableBatchLogs); } + /** + * Method to specify if HTTP requests sent to Splunk should be GZIP encoded. + * + * @param enableGzipHttpCompression whether to enable Gzip encoding. + * @return {@link Builder} + */ + public Builder withEnableGzipHttpCompression(ValueProvider enableGzipHttpCompression) { + return setEnableGzipHttpCompression(enableGzipHttpCompression); + } + /** Build a new {@link SplunkEventWriter} objects based on the configuration. */ public SplunkEventWriter build() { checkNotNull(url(), "url needs to be provided."); diff --git a/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java b/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java index 6f34649946..1173fc9504 100644 --- a/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java +++ b/src/main/java/com/google/cloud/teleport/splunk/SplunkIO.java @@ -81,6 +81,9 @@ public abstract static class Write @Nullable abstract ValueProvider enableBatchLogs(); + @Nullable + abstract ValueProvider enableGzipHttpCompression(); + @Override public PCollection expand(PCollection input) { @@ -92,7 +95,8 @@ public PCollection expand(PCollection input) { .withDisableCertificateValidation(disableCertificateValidation()) .withToken((token())) .withRootCaCertificatePath(rootCaCertificatePath()) - .withEnableBatchLogs(enableBatchLogs()); + .withEnableBatchLogs(enableBatchLogs()) + .withEnableGzipHttpCompression(enableGzipHttpCompression()); SplunkEventWriter writer = builder.build(); LOG.info("SplunkEventWriter configured"); @@ -127,6 +131,9 @@ abstract Builder setDisableCertificateValidation( abstract Builder setEnableBatchLogs(ValueProvider enableBatchLogs); + abstract Builder setEnableGzipHttpCompression( + ValueProvider enableGzipHttpCompression); + abstract Write autoBuild(); /** @@ -295,6 +302,29 @@ public Builder withEnableBatchLogs(Boolean enableBatchLogs) { return setEnableBatchLogs(ValueProvider.StaticValueProvider.of((enableBatchLogs))); } + /** + * Method to specify if HTTP requests sent to Splunk should be GZIP encoded. + * + * @param enableGzipHttpCompression whether to enable Gzip encoding. + * @return {@link Builder} + */ + public Builder withEnableGzipHttpCompression( + ValueProvider enableGzipHttpCompression) { + return setEnableGzipHttpCompression(enableGzipHttpCompression); + } + + /** + * Same as {@link Builder#withEnableGzipHttpCompression(ValueProvider)} but without a {@link + * ValueProvider}. + * + * @param enableGzipHttpCompression whether to enable Gzip encoding. + * @return {@link Builder} + */ + public Builder withEnableGzipHttpCompression(Boolean enableGzipHttpCompression) { + return setEnableGzipHttpCompression( + ValueProvider.StaticValueProvider.of(enableGzipHttpCompression)); + } + public Write build() { checkNotNull(url(), "HEC url is required."); checkNotNull(token(), "Authorization token is required."); diff --git a/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java b/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java index 788b2800e0..2eb57acbb4 100644 --- a/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java +++ b/src/main/java/com/google/cloud/teleport/templates/PubSubToSplunk.java @@ -257,6 +257,7 @@ public static PipelineResult run(PubSubToSplunkOptions options) { .withDisableCertificateValidation(options.getDisableCertificateValidation()) .withRootCaCertificatePath(options.getRootCaCertificatePath()) .withEnableBatchLogs(options.getEnableBatchLogs()) + .withEnableGzipHttpCompression(options.getEnableGzipHttpCompression()) .build()); // 5a) Wrap write failures into a FailsafeElement. diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java index 7bb44bd034..5dbeedd063 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SplunkConverters.java @@ -156,6 +156,13 @@ public interface SplunkOptions extends PipelineOptions { ValueProvider getEnableBatchLogs(); void setEnableBatchLogs(ValueProvider enableBatchLogs); + + @Description( + "Parameter which specifies if HTTP requests sent to Splunk should be GZIP encoded." + + " Default: true.") + ValueProvider getEnableGzipHttpCompression(); + + void setEnableGzipHttpCompression(ValueProvider enableGzipHttpCompression); } private static class FailsafeStringToSplunkEvent diff --git a/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java b/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java index d9b82d528b..9fda460c03 100644 --- a/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java +++ b/src/test/java/com/google/cloud/teleport/splunk/HttpEventPublisherTest.java @@ -104,6 +104,7 @@ public void stringPayloadTest() .withUrl("http://example.com") .withToken("test-token") .withDisableCertificateValidation(false) + .withEnableGzipHttpCompression(true) .build(); String actual = publisher.getStringPayload(SPLUNK_EVENTS); @@ -129,6 +130,7 @@ public void contentTest() .withUrl("http://example.com") .withToken("test-token") .withDisableCertificateValidation(false) + .withEnableGzipHttpCompression(true) .build(); String expectedString = @@ -155,7 +157,8 @@ public void genericURLTest() HttpEventPublisher.newBuilder() .withUrl(baseURL) .withToken("test-token") - .withDisableCertificateValidation(false); + .withDisableCertificateValidation(false) + .withEnableGzipHttpCompression(true); assertThat( builder.genericUrl(), @@ -172,6 +175,7 @@ public void configureBackOffDefaultTest() .withUrl("http://example.com") .withToken("test-token") .withDisableCertificateValidation(false) + .withEnableGzipHttpCompression(true) .build(); assertThat( @@ -191,6 +195,7 @@ public void configureBackOffCustomTest() .withToken("test-token") .withDisableCertificateValidation(false) .withMaxElapsedMillis(timeoutInMillis) + .withEnableGzipHttpCompression(true) .build(); assertThat( @@ -206,6 +211,7 @@ public void invalidRootCaTest() throws Exception { .withToken("test-token") .withDisableCertificateValidation(false) .withRootCaCertificate("invalid_ca".getBytes()) + .withEnableGzipHttpCompression(true) .build(); } @@ -221,6 +227,7 @@ public void recognizedSelfSignedCertificateTest() throws Exception { .withToken("test-token") .withDisableCertificateValidation(false) .withRootCaCertificate(rootCa) + .withEnableGzipHttpCompression(true) .build(); publisher.execute(SPLUNK_EVENTS); @@ -244,6 +251,7 @@ public void unrecognizedSelfSignedCertificateTest() throws Exception { .withToken("test-token") .withDisableCertificateValidation(false) .withRootCaCertificate(rootCa) + .withEnableGzipHttpCompression(true) .build(); publisher.execute(SPLUNK_EVENTS); } From d4aa4d9b966057532051489fd714da86a8546d5e Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Mon, 25 Apr 2022 12:05:06 -0700 Subject: [PATCH 108/145] Update javadoc for Jdbc to Pub/Sub template. PiperOrigin-RevId: 444334851 --- .../com/google/cloud/teleport/v2/templates/JdbcToPubsub.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java index 8d43763892..00928efa58 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToPubsub.java @@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory; /** - * The {@link JdbcToPubsub} streaming pipeline reads data from JdbcIO and publishes to Google Cloud + * The {@link JdbcToPubsub} batch pipeline reads data from JdbcIO and publishes to Google Cloud * PubSub.
*/ public class JdbcToPubsub { From 51724b882f54174d78a86ffedb2d05324e90d5fc Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 25 Apr 2022 13:47:28 -0700 Subject: [PATCH 109/145] Upgrade io.opencensus.opencensus-api to latest version 0.31.0. PiperOrigin-RevId: 444359973 --- v2/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/pom.xml b/v2/pom.xml index e964f02adf..4954518c57 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -37,7 +37,7 @@ 2.6.0 4.13 4.10 - 0.24.0 + 0.31.0 2.21.0 3.0.0 3.6.2 From 63daa31a748729c4e8f9b84a55ebe30205345895 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 26 Apr 2022 22:34:58 -0700 Subject: [PATCH 110/145] Add more unit tests for Spanner Change Streams to BigQuery Dataflow template PiperOrigin-RevId: 444762725 --- .../BigQueryDeadLetterQueueSanitizerTest.java | 71 +++ .../BigQueryDynamicDestinationsTest.java | 151 ++++++ ...lsafeModJsonToTableRowTransformerTest.java | 496 ++++++++++++++++++ .../SchemaUtilsTest.java | 193 +++---- .../TestUtils.java | 187 +++++++ 5 files changed, 1004 insertions(+), 94 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizerTest.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinationsTest.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java create mode 100644 v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizerTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizerTest.java new file mode 100644 index 0000000000..e16558948d --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDeadLetterQueueSanitizerTest.java @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import static com.google.common.truth.Truth.assertThat; + +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse.InsertErrors; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import java.util.Collections; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryInsertError; +import org.codehaus.jackson.node.JsonNodeFactory; +import org.codehaus.jackson.node.ObjectNode; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test class for {@link BigQueryDeadLetterQueueSanitizerTest}. */ +@RunWith(JUnit4.class) +public final class BigQueryDeadLetterQueueSanitizerTest { + + // Test the case where we can get json message and error json message from a failed BigQuery + // insert. + @Test + public void testBigQueryDeadLetterQueueSanitizer() { + TableRow tableRow = new TableRow(); + ObjectNode jsonNode = new ObjectNode(JsonNodeFactory.instance); + jsonNode.put("SingerId", 1); + jsonNode.put("FirstName", "Jack"); + tableRow.set(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_ORIGINAL_PAYLOAD_JSON, jsonNode.toString()); + tableRow.set("SingerId", 1); + tableRow.set("FirstName", "Jack"); + InsertErrors insertErrors = new InsertErrors(); + ErrorProto errorProto = new ErrorProto(); + errorProto.setMessage("Some error message"); + errorProto.setDebugInfo("Some debug info"); + insertErrors.setErrors(Collections.singletonList(errorProto)); + TableReference tableReference = new TableReference(); + tableReference.set( + BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_ORIGINAL_PAYLOAD_JSON, + StandardSQLTypeName.STRING.name()); + tableReference.set("SingerId", StandardSQLTypeName.INT64.name()); + tableReference.set("FirstName", StandardSQLTypeName.STRING.name()); + BigQueryInsertError bigQueryInsertError = + new BigQueryInsertError(tableRow, insertErrors, tableReference); + + assertThat(new BigQueryDeadLetterQueueSanitizer().getJsonMessage(bigQueryInsertError)) + .isEqualTo("{\"SingerId\":1,\"FirstName\":\"Jack\"}"); + assertThat(new BigQueryDeadLetterQueueSanitizer().getErrorMessageJson(bigQueryInsertError)) + .isEqualTo( + "GenericData{classInfo=[errors, index], {errors=[GenericData{classInfo=[debugInfo," + + " location, message, reason], {debugInfo=Some debug info, message=Some error" + + " message}}]}}"); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinationsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinationsTest.java new file mode 100644 index 0000000000..e8fdfe025f --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinationsTest.java @@ -0,0 +1,151 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_BIG_QUERY_DATESET; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_PROJECT; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_SPANNER_CHANGE_STREAM; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_SPANNER_TABLE; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.createSpannerDatabase; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.dropSpannerDatabase; +import static com.google.common.truth.Truth.assertThat; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.teleport.v2.spanner.IntegrationTest; +import com.google.cloud.teleport.v2.spanner.SpannerServerResource; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.BigQueryDynamicDestinations.BigQueryDynamicDestinationsOptions; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test class for {@link BigQueryDynamicDestinationsTest}. */ +@RunWith(JUnit4.class) +@Category(IntegrationTest.class) +public final class BigQueryDynamicDestinationsTest { + + private static BigQueryDynamicDestinations bigQueryDynamicDestinations; + private static TableRow tableRow; + private static KV tableIdToTableRow; + private static String spannerDatabaseName; + + /** Rule for Spanner server resource. */ + @ClassRule public static final SpannerServerResource SPANNER_SERVER = new SpannerServerResource(); + + @BeforeClass + public static void before() throws Exception { + spannerDatabaseName = createSpannerDatabase(SPANNER_SERVER); + BigQueryDynamicDestinationsOptions bigQueryDynamicDestinationsOptions = + BigQueryDynamicDestinationsOptions.builder() + .setSpannerConfig(SPANNER_SERVER.getSpannerConfig(spannerDatabaseName)) + .setChangeStreamName(TEST_SPANNER_CHANGE_STREAM) + .setBigQueryProject(TEST_PROJECT) + .setBigQueryDataset(TEST_BIG_QUERY_DATESET) + .setBigQueryTableTemplate("{_metadata_spanner_table_name}_changelog") + .build(); + bigQueryDynamicDestinations = + BigQueryDynamicDestinations.of(bigQueryDynamicDestinationsOptions); + + tableRow = new TableRow(); + tableRow.set(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_TABLE_NAME, TEST_SPANNER_TABLE); + tableIdToTableRow = + KV.of( + TableId.of(TEST_PROJECT, TEST_BIG_QUERY_DATESET, TEST_SPANNER_TABLE + "_changelog"), + tableRow); + } + + @AfterClass + public static void after() throws Exception { + dropSpannerDatabase(SPANNER_SERVER, spannerDatabaseName); + } + + @Test + public void testGetDestination() { + Instant timestamp = Instant.ofEpochSecond(1649368685L); + PaneInfo paneInfo = PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME); + ValueInSingleWindow tableRowValueInSingleWindow = + ValueInSingleWindow.of(tableRow, timestamp, GlobalWindow.INSTANCE, paneInfo); + TableRow expectedTableRow = new TableRow(); + TableId expectedTableId = + TableId.of(TEST_PROJECT, TEST_BIG_QUERY_DATESET, TEST_SPANNER_TABLE + "_changelog"); + expectedTableRow.set(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_TABLE_NAME, TEST_SPANNER_TABLE); + assertThat(bigQueryDynamicDestinations.getDestination(tableRowValueInSingleWindow)) + .isEqualTo(KV.of(expectedTableId, expectedTableRow)); + } + + @Test + public void testGetTable() { + assertThat(bigQueryDynamicDestinations.getTable(tableIdToTableRow).toString()) + .isEqualTo( + "tableSpec: span-cloud-testing:dataset.AllTypes_changelog tableDescription: BigQuery" + + " changelog table."); + } + + // Test the case where we can get BigQuery schema from Spanner schema which comes from Spanner + // INFORMATION_SCHEMA. + @Test + public void testGetSchema() { + String schemaStr = bigQueryDynamicDestinations.getSchema(tableIdToTableRow).toString(); + schemaStr = + schemaStr.replace( + "classInfo=[categories, collationSpec, description, fields, maxLength, mode, name," + + " policyTags, precision, scale, type], ", + ""); + schemaStr = schemaStr.replace("GenericData", ""); + assertThat(schemaStr) + .isEqualTo( + "{classInfo=[fields], {fields=[{{mode=NULLABLE, name=BooleanPkCol, type=BOOL}}," + + " {{mode=NULLABLE, name=BytesPkCol, type=BYTES}}, {{mode=NULLABLE," + + " name=DatePkCol, type=DATE}}, {{mode=NULLABLE, name=Float64PkCol," + + " type=FLOAT64}}, {{mode=NULLABLE, name=Int64PkCol, type=INT64}}," + + " {{mode=NULLABLE, name=NumericPkCol, type=NUMERIC}}, {{mode=NULLABLE," + + " name=StringPkCol, type=STRING}}, {{mode=NULLABLE, name=TimestampPkCol," + + " type=TIMESTAMP}}, {{mode=REPEATED, name=BooleanArrayCol, type=BOOL}}," + + " {{mode=REPEATED, name=BytesArrayCol, type=BYTES}}, {{mode=REPEATED," + + " name=DateArrayCol, type=DATE}}, {{mode=REPEATED, name=Float64ArrayCol," + + " type=FLOAT64}}, {{mode=REPEATED, name=Int64ArrayCol, type=INT64}}," + + " {{mode=REPEATED, name=JsonArrayCol, type=STRING}}, {{mode=REPEATED," + + " name=NumericArrayCol, type=NUMERIC}}, {{mode=REPEATED, name=StringArrayCol," + + " type=STRING}}, {{mode=REPEATED, name=TimestampArrayCol, type=TIMESTAMP}}," + + " {{mode=NULLABLE, name=BooleanCol, type=BOOL}}, {{mode=NULLABLE, name=BytesCol," + + " type=BYTES}}, {{mode=NULLABLE, name=DateCol, type=DATE}}, {{mode=NULLABLE," + + " name=Float64Col, type=FLOAT64}}, {{mode=NULLABLE, name=Int64Col, type=INT64}}," + + " {{mode=NULLABLE, name=JsonCol, type=STRING}}, {{mode=NULLABLE, name=NumericCol," + + " type=NUMERIC}}, {{mode=NULLABLE, name=StringCol, type=STRING}}," + + " {{mode=NULLABLE, name=TimestampCol, type=TIMESTAMP}}, {{mode=REQUIRED," + + " name=_metadata_spanner_mod_type, type=STRING}}, {{mode=REQUIRED," + + " name=_metadata_spanner_table_name, type=STRING}}, {{mode=REQUIRED," + + " name=_metadata_spanner_commit_timestamp, type=TIMESTAMP}}, {{mode=REQUIRED," + + " name=_metadata_spanner_server_transaction_id, type=STRING}}, {{mode=REQUIRED," + + " name=_metadata_spanner_record_sequence, type=STRING}}, {{mode=REQUIRED," + + " name=_metadata_spanner_is_last_record_in_transaction_in_partition, type=BOOL}}," + + " {{mode=REQUIRED, name=_metadata_spanner_number_of_records_in_transaction," + + " type=INT64}}, {{mode=REQUIRED," + + " name=_metadata_spanner_number_of_partitions_in_transaction, type=INT64}}," + + " {{mode=REQUIRED, name=_metadata_big_query_commit_timestamp," + + " type=TIMESTAMP}}]}}"); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java new file mode 100644 index 0000000000..e024ef4841 --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java @@ -0,0 +1,496 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_SPANNER_CHANGE_STREAM; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_SPANNER_TABLE; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_PK_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_RAW_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.createSpannerDatabase; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.dropSpannerDatabase; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.Statement; +import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.v2.spanner.IntegrationTest; +import com.google.cloud.teleport.v2.spanner.SpannerServerResource; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.Mod; +import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ModType; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.codehaus.jackson.node.ArrayNode; +import org.codehaus.jackson.node.JsonNodeFactory; +import org.codehaus.jackson.node.ObjectNode; +import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test class for {@link FailsafeModJsonToTableRowTransformerTest}. */ +@RunWith(JUnit4.class) +@Category(IntegrationTest.class) +public final class FailsafeModJsonToTableRowTransformerTest { + + /** Rule for Spanner server resource. */ + @ClassRule public static final SpannerServerResource SPANNER_SERVER = new SpannerServerResource(); + + private static String spannerDatabaseName; + private static Timestamp insertCommitTimestamp; + private static Timestamp updateCommitTimestamp; + private static FailsafeModJsonToTableRow failsafeModJsonToTableRow; + + @BeforeClass + public static void before() throws Exception { + spannerDatabaseName = createSpannerDatabase(SPANNER_SERVER); + insertCommitTimestamp = insertRow(spannerDatabaseName); + updateCommitTimestamp = updateRow(spannerDatabaseName); + failsafeModJsonToTableRow = getFailsafeModJsonToTableRow(spannerDatabaseName); + } + + @AfterClass + public static void after() throws Exception { + dropSpannerDatabase(SPANNER_SERVER, spannerDatabaseName); + } + + // Test the case where a TableRow can be constructed from an INSERT Mod. + @Test + public void testFailsafeModJsonToTableRowInsert() throws Exception { + validateBigQueryRow( + spannerDatabaseName, + insertCommitTimestamp, + ModType.INSERT, + getKeysJson(), + getInsertNewValuesJson(insertCommitTimestamp)); + } + + // Test the case where a TableRow can be constructed from a UPDATE Mod. + @Test + public void testFailsafeModJsonToTableRowUpdate() throws Exception { + String updateNewValuesJson = + String.format("{\"TimestampCol\":\"%s\"}", updateCommitTimestamp.toString()); + + validateBigQueryRow( + spannerDatabaseName, + updateCommitTimestamp, + ModType.UPDATE, + getKeysJson(), + updateNewValuesJson); + } + + // Test the case where a TableRow can be constructed from an DELETE Mod. + @Test + public void testFailsafeModJsonToTableRowDelete() throws Exception { + // When we process a mod for deleted row, we only need keys from mod, and don't have to do a + // snapshot read to Spanner, thus we don't need to actually delete the row in Spanner, and we + // can use a fake commit timestamp here. + validateBigQueryRow(spannerDatabaseName, Timestamp.now(), ModType.DELETE, getKeysJson(), ""); + } + + // Test the case where the snapshot read to Spanner fails and we can capture the failures from + // transformDeadLetterOut of FailsafeModJsonToTableRow. + @Test + public void testFailsafeModJsonToTableRowFailedSnapshotRead() throws Exception { + ObjectNode fakePkColJsonNode = new ObjectNode(JsonNodeFactory.instance); + fakePkColJsonNode.put("fakePkCol", true); + ObjectNode fakeNonPkColJsonNode = new ObjectNode(JsonNodeFactory.instance); + fakeNonPkColJsonNode.put("fakeNonPkCol", true); + Mod mod = + new Mod( + fakePkColJsonNode.toString(), + fakeNonPkColJsonNode.toString(), + Timestamp.ofTimeSecondsAndNanos(1650908264L, 925679000), + "1", + true, + "00000001", + TEST_SPANNER_TABLE, + ModType.INSERT, + 1L, + 1L); + TestStream testSream = + TestStream.create(SerializableCoder.of(String.class)) + .addElements(mod.toJson()) + .advanceWatermarkTo(Instant.now()) + .advanceWatermarkToInfinity(); + Pipeline p = Pipeline.create(); + PCollection> input = + p.apply(testSream) + .apply( + ParDo.of( + new DoFn>() { + @ProcessElement + public void process( + @Element String input, + OutputReceiver> receiver) { + receiver.output(FailsafeElement.of(input, input)); + } + })) + .setCoder(SpannerChangeStreamsToBigQuery.FAILSAFE_ELEMENT_CODER); + PCollectionTuple out = input.apply("Mod JSON To TableRow", failsafeModJsonToTableRow); + PAssert.that(out.get(failsafeModJsonToTableRow.transformOut)).empty(); + String expectedPayload = + "{\"keysJson\":\"{\\\"fakePkCol\\\":true}\"," + + "\"newValuesJson\":\"{\\\"fakeNonPkCol\\\":true}\"," + + "\"commitTimestampSeconds\":1650908264,\"commitTimestampNanos\":925679000," + + "\"serverTransactionId\":\"1\",\"isLastRecordInTransactionInPartition\":true," + + "\"recordSequence\":\"00000001\",\"tableName\":\"AllTypes\",\"modType\":\"INSERT\"," + + "\"numberOfRecordsInTransaction\":1,\"numberOfPartitionsInTransaction\":1}"; + PAssert.that( + out.get(failsafeModJsonToTableRow.transformDeadLetterOut) + .apply( + ParDo.of( + new DoFn, String>() { + @ProcessElement + public void process( + @Element FailsafeElement input, + OutputReceiver receiver) { + receiver.output( + String.format( + "originalPayload=%s, payload=%s, errorMessage=%s", + input.getOriginalPayload(), + input.getPayload(), + input.getErrorMessage())); + } + }))) + .containsInAnyOrder( + ImmutableList.of( + String.format( + "originalPayload=%s, payload=%s, errorMessage=Cannot find value for key column" + + " BooleanPkCol", + expectedPayload, expectedPayload))); + p.run().waitUntilFinish(); + } + + private void validateBigQueryRow( + String spannerDatabaseName, + Timestamp commitTimestamp, + ModType modType, + String keysJson, + String newValuesJson) + throws Exception { + Mod mod = + new Mod( + keysJson, + newValuesJson, + commitTimestamp, + "1", + true, + "00000001", + TEST_SPANNER_TABLE, + modType, + 1L, + 1L); + + TableRow expectedTableRow = new TableRow(); + BigQueryUtils.setMetadataFiledsOfTableRow( + TEST_SPANNER_TABLE, mod, mod.toJson(), commitTimestamp, expectedTableRow); + expectedTableRow.set(BOOLEAN_PK_COL, BOOLEAN_RAW_VAL); + expectedTableRow.set(BYTES_PK_COL, BYTES_RAW_VAL.toBase64()); + expectedTableRow.set(DATE_PK_COL, DATE_RAW_VAL.toString()); + expectedTableRow.set(FLOAT64_PK_COL, FLOAT64_RAW_VAL); + expectedTableRow.set(INT64_PK_COL, INT64_RAW_VAL); + expectedTableRow.set(NUMERIC_PK_COL, NUMERIC_RAW_VAL); + expectedTableRow.set(STRING_PK_COL, STRING_RAW_VAL); + expectedTableRow.set(TIMESTAMP_PK_COL, TIMESTAMP_RAW_VAL.toString()); + if (modType == modType.INSERT || modType == modType.UPDATE) { + // The order matters when comparing TableRow, so we need to set different orders for INSERT + // and UPDATE. + if (modType == modType.UPDATE) { + expectedTableRow.set(TIMESTAMP_COL, commitTimestamp.toString()); + } + expectedTableRow.set(BOOLEAN_ARRAY_COL, BOOLEAN_ARRAY_RAW_VAL); + expectedTableRow.set(BYTES_ARRAY_COL, BYTES_ARRAY_RAW_VAL); + expectedTableRow.set(DATE_ARRAY_COL, DATE_ARRAY_RAW_VAL); + expectedTableRow.set(FLOAT64_ARRAY_COL, FLOAT64_ARRAY_RAW_VAL); + expectedTableRow.set(INT64_ARRAY_COL, INT64_ARRAY_RAW_VAL); + expectedTableRow.set(JSON_ARRAY_COL, JSON_ARRAY_RAW_VAL); + expectedTableRow.set(NUMERIC_ARRAY_COL, NUMERIC_ARRAY_RAW_VAL); + expectedTableRow.set(STRING_ARRAY_COL, STRING_ARRAY_RAW_VAL); + expectedTableRow.set(TIMESTAMP_ARRAY_COL, TIMESTAMP_ARRAY_RAW_VAL); + expectedTableRow.set(BOOLEAN_COL, BOOLEAN_RAW_VAL); + expectedTableRow.set(BYTES_COL, BYTES_RAW_VAL.toBase64()); + expectedTableRow.set(DATE_COL, DATE_RAW_VAL.toString()); + expectedTableRow.set(FLOAT64_COL, FLOAT64_RAW_VAL); + expectedTableRow.set(INT64_COL, INT64_RAW_VAL); + expectedTableRow.set(JSON_COL, JSON_RAW_VAL); + expectedTableRow.set(NUMERIC_COL, NUMERIC_RAW_VAL); + expectedTableRow.set(STRING_COL, STRING_RAW_VAL); + if (modType == modType.INSERT) { + expectedTableRow.set(TIMESTAMP_COL, commitTimestamp.toString()); + } + } + + TestStream testSream = + TestStream.create(SerializableCoder.of(String.class)) + .addElements(mod.toJson()) + .advanceWatermarkTo(Instant.now()) + .advanceWatermarkToInfinity(); + Pipeline p = Pipeline.create(); + PCollection> input = + p.apply(testSream) + .apply( + ParDo.of( + new DoFn>() { + @ProcessElement + public void process( + @Element String input, + OutputReceiver> receiver) { + receiver.output(FailsafeElement.of(input, input)); + } + })) + .setCoder(SpannerChangeStreamsToBigQuery.FAILSAFE_ELEMENT_CODER); + PCollectionTuple out = input.apply("Mod JSON To TableRow", failsafeModJsonToTableRow); + PAssert.that( + out.get(failsafeModJsonToTableRow.transformOut) + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void process( + @Element TableRow input, OutputReceiver receiver) { + receiver.output(input.toString()); + } + }))) + .containsInAnyOrder(ImmutableList.of(expectedTableRow.toString())); + PAssert.that(out.get(failsafeModJsonToTableRow.transformDeadLetterOut)).empty(); + p.run().waitUntilFinish(); + } + + private static FailsafeModJsonToTableRow getFailsafeModJsonToTableRow( + String spannerDatabaseName) { + FailsafeModJsonToTableRowOptions failsafeModJsonToTableRowOptions = + FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions.builder() + .setSpannerConfig(SPANNER_SERVER.getSpannerConfig(spannerDatabaseName)) + .setSpannerChangeStream(TEST_SPANNER_CHANGE_STREAM) + .setCoder(SpannerChangeStreamsToBigQuery.FAILSAFE_ELEMENT_CODER) + .build(); + return new FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow( + failsafeModJsonToTableRowOptions); + } + + private static Timestamp insertRow(String spannerDatabaseName) { + List mutations = new ArrayList<>(); + // Set TimestampCol to the commit timestamp, so we can retrieve timestamp by querying this + // column. + // spotless:off + mutations.add( + Mutation.newInsertBuilder(TEST_SPANNER_TABLE) + .set(BOOLEAN_PK_COL).to(BOOLEAN_VAL) + .set(BYTES_PK_COL).to(BYTES_VAL) + .set(DATE_PK_COL).to(DATE_VAL) + .set(FLOAT64_PK_COL).to(FLOAT64_VAL) + .set(INT64_PK_COL).to(INT64_VAL) + .set(NUMERIC_PK_COL).to(NUMERIC_VAL) + .set(STRING_PK_COL).to(STRING_VAL) + .set(TIMESTAMP_PK_COL).to(TIMESTAMP_VAL) + .set(BOOLEAN_ARRAY_COL).to(BOOLEAN_ARRAY_VAL) + .set(BYTES_ARRAY_COL).to(BYTES_ARRAY_VAL) + .set(DATE_ARRAY_COL).to(DATE_ARRAY_VAL) + .set(FLOAT64_ARRAY_COL).to(FLOAT64_ARRAY_VAL) + .set(INT64_ARRAY_COL).to(INT64_ARRAY_VAL) + .set(NUMERIC_ARRAY_COL).to(NUMERIC_ARRAY_VAL) + .set(JSON_ARRAY_COL).to(JSON_ARRAY_VAL) + .set(STRING_ARRAY_COL).to(STRING_ARRAY_VAL) + .set(TIMESTAMP_ARRAY_COL).to(TIMESTAMP_ARRAY_VAL) + .set(BOOLEAN_COL).to(BOOLEAN_VAL) + .set(BYTES_COL).to(BYTES_VAL) + .set(DATE_COL).to(DATE_VAL) + .set(FLOAT64_COL).to(FLOAT64_VAL) + .set(INT64_COL).to(INT64_VAL) + .set(JSON_COL).to(JSON_VAL) + .set(NUMERIC_COL).to(NUMERIC_VAL) + .set(STRING_COL).to(STRING_VAL) + .set(TIMESTAMP_COL).to(Value.COMMIT_TIMESTAMP) + .build()); + // spotless:on + SPANNER_SERVER.getDbClient(spannerDatabaseName).write(mutations); + return getCommitTimestamp(spannerDatabaseName); + } + + private static Timestamp updateRow(String spannerDatabaseName) { + SPANNER_SERVER + .getDbClient(spannerDatabaseName) + .readWriteTransaction() + .run( + transaction -> { + transaction.executeUpdate( + Statement.of( + "UPDATE AllTypes SET TimestampCol = PENDING_COMMIT_TIMESTAMP() WHERE" + + " BooleanPkCol = true")); + return null; + }); + return getCommitTimestamp(spannerDatabaseName); + } + + private static Timestamp getCommitTimestamp(String spannerDatabaseName) { + try (ResultSet resultSet = + SPANNER_SERVER + .getDbClient(spannerDatabaseName) + .singleUse() + .executeQuery(Statement.of("SELECT TimestampCol FROM AllTypes"))) { + while (resultSet.next()) { + return resultSet.getTimestamp(TIMESTAMP_COL); + } + } + throw new RuntimeException("Cannot get commit timestamp from TimestampCol column"); + } + + private String getKeysJson() { + ObjectNode jsonNode = new ObjectNode(JsonNodeFactory.instance); + jsonNode.put(BOOLEAN_PK_COL, BOOLEAN_RAW_VAL); + jsonNode.put(BYTES_PK_COL, BYTES_RAW_VAL.toBase64()); + jsonNode.put(DATE_PK_COL, DATE_RAW_VAL.toString()); + jsonNode.put(FLOAT64_PK_COL, FLOAT64_RAW_VAL); + jsonNode.put(INT64_PK_COL, INT64_RAW_VAL); + jsonNode.put(NUMERIC_PK_COL, NUMERIC_RAW_VAL); + jsonNode.put(STRING_PK_COL, STRING_RAW_VAL); + jsonNode.put(TIMESTAMP_PK_COL, TIMESTAMP_RAW_VAL.toString()); + return jsonNode.toString(); + } + + private String getInsertNewValuesJson(Timestamp insertCommitTimestamp) { + ObjectNode jsonNode = new ObjectNode(JsonNodeFactory.instance); + ArrayNode arrayNode = jsonNode.putArray(BOOLEAN_ARRAY_COL); + arrayNode.add(BOOLEAN_ARRAY_RAW_VAL.get(0)); + arrayNode.add(BOOLEAN_ARRAY_RAW_VAL.get(1)); + arrayNode.add(BOOLEAN_ARRAY_RAW_VAL.get(2)); + arrayNode = jsonNode.putArray(BYTES_ARRAY_COL); + arrayNode.add(BYTES_ARRAY_RAW_VAL.get(0)); + arrayNode.add(BYTES_ARRAY_RAW_VAL.get(1)); + arrayNode.add(BYTES_ARRAY_RAW_VAL.get(2)); + arrayNode = jsonNode.putArray(DATE_ARRAY_COL); + arrayNode.add(DATE_ARRAY_RAW_VAL.get(0).toString()); + arrayNode.add(DATE_ARRAY_RAW_VAL.get(1).toString()); + arrayNode = jsonNode.putArray(FLOAT64_ARRAY_COL); + arrayNode.add(FLOAT64_ARRAY_RAW_VAL.get(0)); + arrayNode.add(FLOAT64_ARRAY_RAW_VAL.get(1)); + arrayNode.add(FLOAT64_ARRAY_RAW_VAL.get(2)); + arrayNode.add(FLOAT64_ARRAY_RAW_VAL.get(3)); + arrayNode.add(FLOAT64_ARRAY_RAW_VAL.get(4)); + arrayNode.add(FLOAT64_ARRAY_RAW_VAL.get(5)); + arrayNode = jsonNode.putArray(INT64_ARRAY_COL); + arrayNode.add(INT64_ARRAY_RAW_VAL.get(0)); + arrayNode.add(INT64_ARRAY_RAW_VAL.get(1)); + arrayNode.add(INT64_ARRAY_RAW_VAL.get(2)); + arrayNode.add(INT64_ARRAY_RAW_VAL.get(3)); + arrayNode.add(INT64_ARRAY_RAW_VAL.get(4)); + arrayNode = jsonNode.putArray(JSON_ARRAY_COL); + arrayNode.add(JSON_ARRAY_RAW_VAL.get(0)); + arrayNode.add(JSON_ARRAY_RAW_VAL.get(1)); + arrayNode.add(JSON_ARRAY_RAW_VAL.get(2)); + arrayNode = jsonNode.putArray(NUMERIC_ARRAY_COL); + arrayNode.add(NUMERIC_ARRAY_RAW_VAL.get(0)); + arrayNode.add(NUMERIC_ARRAY_RAW_VAL.get(1)); + arrayNode.add(NUMERIC_ARRAY_RAW_VAL.get(2)); + arrayNode = jsonNode.putArray(STRING_ARRAY_COL); + arrayNode.add(STRING_ARRAY_RAW_VAL.get(0)); + arrayNode.add(STRING_ARRAY_RAW_VAL.get(1)); + arrayNode.add(STRING_ARRAY_RAW_VAL.get(2)); + arrayNode = jsonNode.putArray(TIMESTAMP_ARRAY_COL); + arrayNode.add(TIMESTAMP_ARRAY_RAW_VAL.get(0).toString()); + arrayNode.add(TIMESTAMP_ARRAY_RAW_VAL.get(1).toString()); + arrayNode.add(TIMESTAMP_ARRAY_RAW_VAL.get(2).toString()); + jsonNode.put(BOOLEAN_COL, BOOLEAN_RAW_VAL); + jsonNode.put(BYTES_COL, BYTES_RAW_VAL.toBase64()); + jsonNode.put(DATE_COL, DATE_RAW_VAL.toString()); + jsonNode.put(FLOAT64_COL, FLOAT64_RAW_VAL); + jsonNode.put(INT64_COL, INT64_RAW_VAL); + jsonNode.put(JSON_COL, JSON_RAW_VAL); + jsonNode.put(NUMERIC_COL, NUMERIC_RAW_VAL); + jsonNode.put(STRING_COL, STRING_RAW_VAL); + jsonNode.put(TIMESTAMP_COL, insertCommitTimestamp.toString()); + return jsonNode.toString(); + } +} diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java index d50ab63354..61d6c6875b 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java @@ -15,13 +15,46 @@ */ package com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_VAL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_VAL; import static com.google.common.truth.Truth.assertThat; import static org.mockito.Mockito.when; import com.google.api.services.bigquery.model.TableRow; -import com.google.cloud.ByteArray; -import com.google.cloud.Date; -import com.google.cloud.Timestamp; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.Key; import com.google.cloud.spanner.ReadContext; @@ -36,9 +69,7 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; import com.google.common.collect.ImmutableList; -import java.math.BigDecimal; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -64,24 +95,24 @@ public void setUp() { when(mockDatabaseClient.singleUse()).thenReturn(mockReadContext); spannerColumnsOfAllTypes = ImmutableList.of( - TrackedSpannerColumn.create("BoolCol", Type.bool(), 1), - TrackedSpannerColumn.create("BytesCol", Type.bytes(), 2), - TrackedSpannerColumn.create("DateCol", Type.date(), 3), - TrackedSpannerColumn.create("Float64Col", Type.float64(), 4), - TrackedSpannerColumn.create("Int64Col", Type.int64(), 5), - TrackedSpannerColumn.create("JsonCol", Type.json(), 6), - TrackedSpannerColumn.create("NumericCol", Type.numeric(), 7), - TrackedSpannerColumn.create("StringCol", Type.string(), 8), - TrackedSpannerColumn.create("TimestampCol", Type.timestamp(), 9), - TrackedSpannerColumn.create("BoolArrayCol", Type.array(Type.bool()), 10), - TrackedSpannerColumn.create("BytesArrayCol", Type.array(Type.bytes()), 11), - TrackedSpannerColumn.create("DateArrayCol", Type.array(Type.date()), 12), - TrackedSpannerColumn.create("Float64ArrayCol", Type.array(Type.float64()), 13), - TrackedSpannerColumn.create("Int64ArrayCol", Type.array(Type.int64()), 14), - TrackedSpannerColumn.create("JsonArrayCol", Type.array(Type.json()), 15), - TrackedSpannerColumn.create("NumericArrayCol", Type.array(Type.numeric()), 16), - TrackedSpannerColumn.create("StringArrayCol", Type.array(Type.string()), 17), - TrackedSpannerColumn.create("TimestampArrayCol", Type.array(Type.timestamp()), 18)); + TrackedSpannerColumn.create(BOOLEAN_COL, Type.bool(), 1), + TrackedSpannerColumn.create(BYTES_COL, Type.bytes(), 2), + TrackedSpannerColumn.create(DATE_COL, Type.date(), 3), + TrackedSpannerColumn.create(FLOAT64_COL, Type.float64(), 4), + TrackedSpannerColumn.create(INT64_COL, Type.int64(), 5), + TrackedSpannerColumn.create(JSON_COL, Type.json(), 6), + TrackedSpannerColumn.create(NUMERIC_COL, Type.numeric(), 7), + TrackedSpannerColumn.create(STRING_COL, Type.string(), 8), + TrackedSpannerColumn.create(TIMESTAMP_COL, Type.timestamp(), 9), + TrackedSpannerColumn.create(BOOLEAN_ARRAY_COL, Type.array(Type.bool()), 10), + TrackedSpannerColumn.create(BYTES_ARRAY_COL, Type.array(Type.bytes()), 11), + TrackedSpannerColumn.create(DATE_ARRAY_COL, Type.array(Type.date()), 12), + TrackedSpannerColumn.create(FLOAT64_ARRAY_COL, Type.array(Type.float64()), 13), + TrackedSpannerColumn.create(INT64_ARRAY_COL, Type.array(Type.int64()), 14), + TrackedSpannerColumn.create(JSON_ARRAY_COL, Type.array(Type.json()), 15), + TrackedSpannerColumn.create(NUMERIC_ARRAY_COL, Type.array(Type.numeric()), 16), + TrackedSpannerColumn.create(STRING_ARRAY_COL, Type.array(Type.string()), 17), + TrackedSpannerColumn.create(TIMESTAMP_ARRAY_COL, Type.array(Type.timestamp()), 18)); } @Test @@ -159,6 +190,7 @@ public void testChangeStreamTrackTwoColumns() { String sql = "SELECT TABLE_NAME, COLUMN_NAME FROM INFORMATION_SCHEMA.CHANGE_STREAM_COLUMNS " + "WHERE CHANGE_STREAM_NAME = @changeStreamName"; + // spotless:off when(mockReadContext.executeQuery( Statement.newBuilder(sql).bind("changeStreamName").to(changeStreamName).build())) .thenReturn( @@ -179,6 +211,7 @@ public void testChangeStreamTrackTwoColumns() { .set("COLUMN_NAME") .to(Value.string("FirstName")) .build()))); + // spotless:on Map actualSpannerTableByName = new SpannerUtils(mockDatabaseClient, changeStreamName).getSpannerTableByName(); @@ -196,27 +229,6 @@ public void testChangeStreamTrackTwoColumns() { @Test public void testSpannerSnapshotRowToBigQueryTableRow() { TableRow tableRow = new TableRow(); - Boolean[] booleanArray = {true, false, true}; - ByteArray[] bytesArray = { - ByteArray.copyFrom("123"), ByteArray.copyFrom("456"), ByteArray.copyFrom("789") - }; - Date[] dateArray = {Date.fromYearMonthDay(2022, 1, 22), Date.fromYearMonthDay(2022, 3, 11)}; - double[] float64Array = {Double.MIN_VALUE, Double.MAX_VALUE, 0, 1, -1, 1.2341}; - long[] int64Array = {Long.MAX_VALUE, Long.MIN_VALUE, 0, 1, -1}; - String[] jsonArray = {"{}", "{\"color\":\"red\",\"value\":\"#f00\"}", "[]"}; - BigDecimal[] numericArray = { - BigDecimal.valueOf(1, Integer.MAX_VALUE), - BigDecimal.valueOf(1, Integer.MIN_VALUE), - BigDecimal.ZERO, - BigDecimal.TEN, - BigDecimal.valueOf(3141592, 6) - }; - String[] stringArray = {"abc", "def", "ghi"}; - Timestamp[] timestampArray = { - Timestamp.ofTimeSecondsAndNanos(1646617853L, 972000000), - Timestamp.ofTimeSecondsAndNanos(1646637853L, 572000000), - Timestamp.ofTimeSecondsAndNanos(1646657853L, 772000000) - }; List structFields = new ArrayList<>(spannerColumnsOfAllTypes.size()); for (TrackedSpannerColumn spannerColumn : spannerColumnsOfAllTypes) { structFields.add(Type.StructField.of(spannerColumn.getName(), spannerColumn.getType())); @@ -227,25 +239,24 @@ public void testSpannerSnapshotRowToBigQueryTableRow() { Type.struct(structFields), Collections.singletonList( Struct.newBuilder() - .set("BoolCol").to(Value.bool(true)) - .set("BytesCol").to(Value.bytes(ByteArray.copyFrom("123"))) - .set("DateCol").to(Date.fromYearMonthDay(2022, 1, 22)) - .set("Float64Col").to(Value.float64(1.2)) - .set("Int64Col").to(Value.int64(20)) - .set("JsonCol").to(Value.json("{\"color\":\"red\",\"value\":\"#f00\"}")) - .set("NumericCol").to(Value.numeric(BigDecimal.valueOf(123, 2))) - .set("StringCol").to(Value.string("abc")) - .set("TimestampCol").to(Timestamp.ofTimeSecondsAndNanos(1646617853L, 972000000)) - .set("BoolArrayCol").to(Value.boolArray(Arrays.asList(booleanArray))) - .set("BytesArrayCol").to(Value.bytesArray(Arrays.asList(bytesArray))) - .set("DateArrayCol").to(Value.dateArray(Arrays.asList(dateArray))) - .set("Float64ArrayCol").to(Value.float64Array(float64Array)) - .set("Int64ArrayCol").to(Value.int64Array(int64Array)) - .set("JsonArrayCol").to(Value.jsonArray(Arrays.asList(jsonArray))) - .set("NumericArrayCol").to(Value.numericArray(Arrays.asList(numericArray))) - .set("StringArrayCol").to(Value.stringArray(Arrays.asList(stringArray))) - .set("TimestampArrayCol") - .to(Value.timestampArray(Arrays.asList(timestampArray))) + .set(BOOLEAN_COL).to(BOOLEAN_VAL) + .set(BYTES_COL).to(BYTES_VAL) + .set(DATE_COL).to(DATE_VAL) + .set(FLOAT64_COL).to(FLOAT64_VAL) + .set(INT64_COL).to(INT64_VAL) + .set(JSON_COL).to(JSON_VAL) + .set(NUMERIC_COL).to(NUMERIC_VAL) + .set(STRING_COL).to(STRING_VAL) + .set(TIMESTAMP_COL).to(TIMESTAMP_VAL) + .set(BOOLEAN_ARRAY_COL).to(BOOLEAN_ARRAY_VAL) + .set(BYTES_ARRAY_COL).to(BYTES_ARRAY_VAL) + .set(DATE_ARRAY_COL).to(DATE_ARRAY_VAL) + .set(FLOAT64_ARRAY_COL).to(FLOAT64_ARRAY_VAL) + .set(INT64_ARRAY_COL).to(INT64_ARRAY_VAL) + .set(JSON_ARRAY_COL).to(JSON_ARRAY_VAL) + .set(NUMERIC_ARRAY_COL).to(NUMERIC_ARRAY_VAL) + .set(STRING_ARRAY_COL).to(STRING_ARRAY_VAL) + .set(TIMESTAMP_ARRAY_COL).to(TIMESTAMP_ARRAY_VAL) .build())); // spotless:on SpannerToBigQueryUtils.spannerSnapshotRowToBigQueryTableRow( @@ -253,16 +264,15 @@ public void testSpannerSnapshotRowToBigQueryTableRow() { assertThat(tableRow.toString()) .isEqualTo( - "GenericData{classInfo=[f], {BoolCol=true, BytesCol=MTIz, DateCol=2022-01-22," - + " Float64Col=1.2, Int64Col=20, JsonCol={\"color\":\"red\",\"value\":\"#f00\"}," - + " NumericCol=1.23, StringCol=abc, TimestampCol=2022-03-07T01:50:53.972000000Z," - + " BoolArrayCol=[true, false, true], BytesArrayCol=[MTIz, NDU2, Nzg5]," + "GenericData{classInfo=[f], {BooleanCol=true, BytesCol=NDU2, DateCol=2022-03-11," + + " Float64Col=2.5, Int64Col=10, JsonCol={\"color\":\"red\"}, NumericCol=10," + + " StringCol=abc, TimestampCol=2022-03-07T01:50:53.972000000Z," + + " BooleanArrayCol=[true, false, true], BytesArrayCol=[MTIz, NDU2, Nzg5]," + " DateArrayCol=[2022-01-22, 2022-03-11], Float64ArrayCol=[4.9E-324," + " 1.7976931348623157E308, 0.0, 1.0, -1.0, 1.2341]," + " Int64ArrayCol=[9223372036854775807, -9223372036854775808, 0, 1, -1]," + " JsonArrayCol=[{}, {\"color\":\"red\",\"value\":\"#f00\"}, []]," - + " NumericArrayCol=[1E-2147483647, 1E+2147483648, 0, 10, 3.141592]," - + " StringArrayCol=[abc, def, ghi]," + + " NumericArrayCol=[0, 10, 3.141592], StringArrayCol=[abc, def, ghi]," + " TimestampArrayCol=[2022-03-07T01:50:53.972000000Z," + " 2022-03-07T07:24:13.572000000Z, 2022-03-07T12:57:33.772000000Z]}}"); } @@ -270,14 +280,14 @@ public void testSpannerSnapshotRowToBigQueryTableRow() { @Test public void testAppendToSpannerKey() { JSONObject keysJsonObject = new JSONObject(); - keysJsonObject.put("BoolCol", true); - keysJsonObject.put("BytesCol", ByteArray.copyFrom("123").toBase64()); - keysJsonObject.put("DateCol", "2022-01-22"); - keysJsonObject.put("Float64Col", 1.2); - keysJsonObject.put("Int64Col", 20); - keysJsonObject.put("NumericCol", new BigDecimal(3.141592)); - keysJsonObject.put("StringCol", "abc"); - keysJsonObject.put("TimestampCol", "2022-03-07T01:50:53.972000000Z"); + keysJsonObject.put(BOOLEAN_COL, BOOLEAN_VAL.getBool()); + keysJsonObject.put(BYTES_COL, BYTES_VAL.getBytes().toBase64()); + keysJsonObject.put(DATE_COL, DATE_VAL.getDate().toString()); + keysJsonObject.put(FLOAT64_COL, FLOAT64_VAL.getFloat64()); + keysJsonObject.put(INT64_COL, INT64_VAL.getInt64()); + keysJsonObject.put(NUMERIC_COL, NUMERIC_VAL.getNumeric()); + keysJsonObject.put(STRING_COL, STRING_VAL.getString()); + keysJsonObject.put(TIMESTAMP_COL, TIMESTAMP_VAL.toString()); Key.Builder keyBuilder = com.google.cloud.spanner.Key.newBuilder(); for (TrackedSpannerColumn spannerColumn : spannerColumnsOfAllTypes) { String typeName = spannerColumn.getType().getCode().name(); @@ -289,10 +299,7 @@ public void testAppendToSpannerKey() { } assertThat(keyBuilder.build().toString()) - .isEqualTo( - "[true,MTIz,2022-01-22,1.2,20," - + "3.14159200000000016217427400988526642322540283203125,abc," - + "2022-03-07T01:50:53.972000000Z]"); + .isEqualTo("[true,NDU2,2022-03-11,2.5,10,10,abc,2022-03-07T01:50:53.972000000Z]"); } @Test @@ -310,7 +317,7 @@ public void testSpannerColumnsToBigQueryIOFields() { assertThat(bigQueryIOFieldsStr) .isEqualTo( - "[{{mode=NULLABLE, name=BoolCol, type=BOOL}}, " + "[{{mode=NULLABLE, name=BooleanCol, type=BOOL}}, " + "{{mode=NULLABLE, name=BytesCol, type=BYTES}}, " + "{{mode=NULLABLE, name=DateCol, type=DATE}}, " + "{{mode=NULLABLE, name=Float64Col, type=FLOAT64}}, " @@ -319,7 +326,7 @@ public void testSpannerColumnsToBigQueryIOFields() { + "{{mode=NULLABLE, name=NumericCol, type=NUMERIC}}, " + "{{mode=NULLABLE, name=StringCol, type=STRING}}, " + "{{mode=NULLABLE, name=TimestampCol, type=TIMESTAMP}}, " - + "{{mode=REPEATED, name=BoolArrayCol, type=BOOL}}, " + + "{{mode=REPEATED, name=BooleanArrayCol, type=BOOL}}, " + "{{mode=REPEATED, name=BytesArrayCol, type=BYTES}}, " + "{{mode=REPEATED, name=DateArrayCol, type=DATE}}, " + "{{mode=REPEATED, name=Float64ArrayCol, type=FLOAT64}}, " @@ -331,7 +338,7 @@ public void testSpannerColumnsToBigQueryIOFields() { } @Test - public void testAddSpannerNonPkColumnsToTableRow() { + public void testAddSpannerNonPkColumnsToTableRow() throws Exception { String newValuesJson = "{\"BoolCol\":true,\"BytesCol\":\"ZmZm\",\"DateCol\":\"2020-12-12\",\"Float64Col\":1.3," + "\"Int64Col\":\"5\"," @@ -352,16 +359,14 @@ public void testAddSpannerNonPkColumnsToTableRow() { assertThat(tableRow.toString()) .isEqualTo( - "GenericData{classInfo=[f], " - + "{BoolCol=true, BytesCol=ZmZm, DateCol=2020-12-12, Float64Col=1.3, Int64Col=5, " - + "JsonCol={\"color\":\"red\",\"value\":\"#f00\"}, NumericCol=4.4, StringCol=abc, " - + "TimestampCol=2022-03-19T18:51:33.963910279Z, BoolArrayCol=[true, false], " - + "BytesArrayCol=[YWJj, YmNk], DateArrayCol=[2021-01-22, 2022-01-01], " - + "Float64ArrayCol=[1.2, 4.4], Int64ArrayCol=[1, 2], " - + "JsonArrayCol=[{}, {\"color\":\"red\",\"value\":\"#f00\"}, []], " - + "NumericArrayCol=[2.2, 3.3], StringArrayCol=[a, b], " - + "TimestampArrayCol=[2022-03-19T18:51:33.963910279Z, " - + "2022-03-19T18:51:33.963910279Z]}}"); + "GenericData{classInfo=[f], {BytesCol=ZmZm, DateCol=2020-12-12, Float64Col=1.3," + + " Int64Col=5, JsonCol={\"color\":\"red\",\"value\":\"#f00\"}, NumericCol=4.4," + + " StringCol=abc, TimestampCol=2022-03-19T18:51:33.963910279Z," + + " BytesArrayCol=[YWJj, YmNk], DateArrayCol=[2021-01-22, 2022-01-01]," + + " Float64ArrayCol=[1.2, 4.4], Int64ArrayCol=[1, 2], JsonArrayCol=[{}," + + " {\"color\":\"red\",\"value\":\"#f00\"}, []], NumericArrayCol=[2.2, 3.3]," + + " StringArrayCol=[a, b], TimestampArrayCol=[2022-03-19T18:51:33.963910279Z," + + " 2022-03-19T18:51:33.963910279Z]}}"); } private void mockInformationSchemaChangeStreamsQuery(boolean isTrackingAll) { diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java new file mode 100644 index 0000000000..c4c14a9fdf --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java @@ -0,0 +1,187 @@ +/* + * Copyright (C) 2022 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.v2.templates.spannerchangestreamstobigquery; + +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.v2.spanner.SpannerServerResource; +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; +import org.apache.commons.lang3.RandomStringUtils; + +/** {@link TestUtils} provides methods for testing. */ +class TestUtils { + + static final String TEST_PROJECT = "span-cloud-testing"; + static final String TEST_SPANNER_INSTANCE = "changestream"; + static final String TEST_SPANNER_DATABASE_PREFIX = "testdbchangestreams"; + static final String TEST_SPANNER_HOST = "https://spanner.googleapis.com"; + static final String TEST_SPANNER_TABLE = "AllTypes"; + static final String TEST_SPANNER_CHANGE_STREAM = "AllTypesStream"; + static final String TEST_BIG_QUERY_DATESET = "dataset"; + static final int MAX_TABLE_NAME_LENGTH = 29; + + static final String BOOLEAN_PK_COL = "BooleanPkCol"; + static final String BYTES_PK_COL = "BytesPkCol"; + static final String DATE_PK_COL = "DatePkCol"; + static final String FLOAT64_PK_COL = "Float64PkCol"; + static final String INT64_PK_COL = "Int64PkCol"; + static final String NUMERIC_PK_COL = "NumericPkCol"; + static final String STRING_PK_COL = "StringPkCol"; + static final String TIMESTAMP_PK_COL = "TimestampPkCol"; + + static final String BOOLEAN_ARRAY_COL = "BooleanArrayCol"; + static final String BYTES_ARRAY_COL = "BytesArrayCol"; + static final String DATE_ARRAY_COL = "DateArrayCol"; + static final String FLOAT64_ARRAY_COL = "Float64ArrayCol"; + static final String INT64_ARRAY_COL = "Int64ArrayCol"; + static final String NUMERIC_ARRAY_COL = "NumericArrayCol"; + static final String JSON_ARRAY_COL = "JsonArrayCol"; + static final String STRING_ARRAY_COL = "StringArrayCol"; + static final String TIMESTAMP_ARRAY_COL = "TimestampArrayCol"; + + static final String BOOLEAN_COL = "BooleanCol"; + static final String BYTES_COL = "BytesCol"; + static final String DATE_COL = "DateCol"; + static final String FLOAT64_COL = "Float64Col"; + static final String INT64_COL = "Int64Col"; + static final String JSON_COL = "JsonCol"; + static final String NUMERIC_COL = "NumericCol"; + static final String STRING_COL = "StringCol"; + static final String TIMESTAMP_COL = "TimestampCol"; + + static final Boolean BOOLEAN_RAW_VAL = true; + static final ByteArray BYTES_RAW_VAL = ByteArray.copyFrom("456"); + static final Date DATE_RAW_VAL = Date.fromYearMonthDay(2022, 3, 11); + static final Double FLOAT64_RAW_VAL = 2.5; + static final Long INT64_RAW_VAL = 10L; + static final String JSON_RAW_VAL = "{\"color\":\"red\"}"; + static final BigDecimal NUMERIC_RAW_VAL = BigDecimal.TEN; + static final String STRING_RAW_VAL = "abc"; + static final Timestamp TIMESTAMP_RAW_VAL = + Timestamp.ofTimeSecondsAndNanos(1646617853L, 972000000); + + static final Value BOOLEAN_VAL = Value.bool(BOOLEAN_RAW_VAL); + static final Value BYTES_VAL = Value.bytes(BYTES_RAW_VAL); + static final Value DATE_VAL = Value.date(DATE_RAW_VAL); + static final Value FLOAT64_VAL = Value.float64(FLOAT64_RAW_VAL); + static final Value INT64_VAL = Value.int64(INT64_RAW_VAL); + static final Value JSON_VAL = Value.json(JSON_RAW_VAL); + static final Value NUMERIC_VAL = Value.numeric(NUMERIC_RAW_VAL); + static final Value STRING_VAL = Value.string(STRING_RAW_VAL); + static final Value TIMESTAMP_VAL = Value.timestamp(TIMESTAMP_RAW_VAL); + + static final List BOOLEAN_ARRAY_RAW_VAL = Arrays.asList(true, false, true); + static final List BYTES_ARRAY_RAW_VAL = + Arrays.asList( + ByteArray.copyFrom("123").toBase64(), + ByteArray.copyFrom("456").toBase64(), + ByteArray.copyFrom("789").toBase64()); + static final List DATE_ARRAY_RAW_VAL = + Arrays.asList(Date.fromYearMonthDay(2022, 1, 22), Date.fromYearMonthDay(2022, 3, 11)); + static final List FLOAT64_ARRAY_RAW_VAL = + Arrays.asList(Double.MIN_VALUE, Double.MAX_VALUE, 0.0, 1.0, -1.0, 1.2341); + static final List INT64_ARRAY_RAW_VAL = + Arrays.asList(Long.MAX_VALUE, Long.MIN_VALUE, 0L, 1L, -1L); + static final List JSON_ARRAY_RAW_VAL = + Arrays.asList("{}", "{\"color\":\"red\",\"value\":\"#f00\"}", "[]"); + static final List NUMERIC_ARRAY_RAW_VAL = + Arrays.asList(BigDecimal.ZERO, BigDecimal.TEN, BigDecimal.valueOf(3141592, 6)); + static final List STRING_ARRAY_RAW_VAL = Arrays.asList("abc", "def", "ghi"); + static final List TIMESTAMP_ARRAY_RAW_VAL = + Arrays.asList( + Timestamp.ofTimeSecondsAndNanos(1646617853L, 972000000), + Timestamp.ofTimeSecondsAndNanos(1646637853L, 572000000), + Timestamp.ofTimeSecondsAndNanos(1646657853L, 772000000)); + + static final Value BOOLEAN_ARRAY_VAL = Value.boolArray(BOOLEAN_ARRAY_RAW_VAL); + static final Value BYTES_ARRAY_VAL = + Value.bytesArray( + Arrays.asList( + ByteArray.copyFrom("123"), ByteArray.copyFrom("456"), ByteArray.copyFrom("789"))); + static final Value DATE_ARRAY_VAL = Value.dateArray(DATE_ARRAY_RAW_VAL); + static final Value FLOAT64_ARRAY_VAL = Value.float64Array(FLOAT64_ARRAY_RAW_VAL); + static final Value INT64_ARRAY_VAL = Value.int64Array(INT64_ARRAY_RAW_VAL); + static final Value JSON_ARRAY_VAL = Value.jsonArray(JSON_ARRAY_RAW_VAL); + static final Value NUMERIC_ARRAY_VAL = Value.numericArray(NUMERIC_ARRAY_RAW_VAL); + static final Value STRING_ARRAY_VAL = Value.stringArray(STRING_ARRAY_RAW_VAL); + static final Value TIMESTAMP_ARRAY_VAL = Value.timestampArray(TIMESTAMP_ARRAY_RAW_VAL); + + private static String generateSpannerDatabaseName() { + return TEST_SPANNER_DATABASE_PREFIX + + "_" + + RandomStringUtils.randomNumeric( + MAX_TABLE_NAME_LENGTH - 1 - TEST_SPANNER_DATABASE_PREFIX.length()); + } + + // This methods does the following: + // 1. Generate a random database name. + // 2. Drop the database with the generated name if it already exists. + // 3. Create a database with the generated name, the database has one table with columns of all + // supported Spanner types, and it contains a change stream that watches this table. + public static String createSpannerDatabase(SpannerServerResource spannerServer) throws Exception { + String spannerDatabaseName = generateSpannerDatabaseName(); + spannerServer.dropDatabase(spannerDatabaseName); + + // spotless:off + String createTableDdl = + "CREATE TABLE " + + TEST_SPANNER_TABLE + + " (" + + "BooleanPkCol BOOL," + + "BytesPkCol BYTES(1024)," + + "DatePkCol DATE," + + "Float64PkCol FLOAT64," + + "Int64PkCol INT64," + + "NumericPkCol NUMERIC," + + "StringPkCol STRING(MAX)," + + "TimestampPkCol TIMESTAMP OPTIONS (allow_commit_timestamp=true)," + + "BooleanArrayCol ARRAY," + + "BytesArrayCol ARRAY," + + "DateArrayCol ARRAY," + + "Float64ArrayCol ARRAY," + + "Int64ArrayCol ARRAY," + + "JsonArrayCol ARRAY," + + "NumericArrayCol ARRAY," + + "StringArrayCol ARRAY," + + "TimestampArrayCol ARRAY," + + "BooleanCol BOOL," + + "BytesCol BYTES(1024)," + + "DateCol DATE," + + "Float64Col FLOAT64," + + "Int64Col INT64," + + "JsonCol JSON," + + "NumericCol NUMERIC," + + "StringCol STRING(1024)," + + "TimestampCol TIMESTAMP OPTIONS (allow_commit_timestamp=true)" + + ") PRIMARY KEY(BooleanPkCol, BytesPkCol, DatePkCol, Float64PkCol, Int64PkCol," + + " NumericPkCol, StringPkCol, TimestampPkCol)"; + // spotless:on + String createChangeStreamDdl = + "CREATE CHANGE STREAM " + TEST_SPANNER_CHANGE_STREAM + " FOR " + TEST_SPANNER_TABLE; + List statements = Arrays.asList(createTableDdl, createChangeStreamDdl); + spannerServer.createDatabase(spannerDatabaseName, statements); + return spannerDatabaseName; + } + + public static void dropSpannerDatabase( + SpannerServerResource spannerServer, String spannerDatabaseName) { + spannerServer.dropDatabase(spannerDatabaseName); + } +} From a886c3ce6c09d764dae47e6f4b752ace91c9f57e Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Wed, 27 Apr 2022 10:01:53 -0700 Subject: [PATCH 111/145] Implement ignoreFields for Cloud Spanner Change Streams to BigQuery Dataflow template PiperOrigin-RevId: 444891248 --- ...SpannerChangeStreamsToBigQueryOptions.java | 15 ++++++++++---- .../FailsafeModJsonToTableRowTransformer.java | 20 ++++++++++++++++++- .../SpannerChangeStreamsToBigQuery.java | 2 +- ...lsafeModJsonToTableRowTransformerTest.java | 1 + 4 files changed, 32 insertions(+), 6 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java index 026547abb9..d9ae1d9435 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java @@ -118,10 +118,17 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setDlqRetryMinutes(Integer value); - // TODO(haikuo-google): Implement this functionality. - @Description("Comma Separated list of metadata fields to be ignored") + @Description( + "Comma separated list of fields to be ignored, these could be fields of tracked tables, or" + + " metadata fields which are _metadata_spanner_mod_type, _metadata_spanner_table_name," + + " _metadata_spanner_commit_timestamp, _metadata_spanner_server_transaction_id," + + " _metadata_spanner_record_sequence," + + " _metadata_spanner_is_last_record_in_transaction_in_partition," + + " _metadata_spanner_number_of_records_in_transaction," + + " _metadata_spanner_number_of_partitions_in_transaction," + + " _metadata_big_query_commit_timestamp") @Default.String("") - String getIgnoreMetadataFields(); + String getIgnoreFields(); - void setIgnoreMetadataFields(String value); + void setIgnoreFields(String value); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java index 0da3be3516..a8b358c621 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java @@ -31,8 +31,10 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; import com.google.cloud.teleport.v2.values.FailsafeElement; import java.io.Serializable; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; @@ -83,6 +85,7 @@ public PCollectionTuple expand(PCollection> inpu new FailsafeModJsonToTableRowFn( failsafeModJsonToTableRowOptions.getSpannerConfig(), failsafeModJsonToTableRowOptions.getSpannerChangeStream(), + failsafeModJsonToTableRowOptions.getIgnoreFields(), transformOut, transformDeadLetterOut)) .withOutputTags(transformOut, TupleTagList.of(transformDeadLetterOut))); @@ -100,19 +103,25 @@ public static class FailsafeModJsonToTableRowFn private transient SpannerAccessor spannerAccessor; private final SpannerConfig spannerConfig; private final String spannerChangeStream; - private Map spannerTableByName = null; + private Map spannerTableByName; + private final Set ignoreFields; public TupleTag transformOut; public TupleTag> transformDeadLetterOut; public FailsafeModJsonToTableRowFn( SpannerConfig spannerConfig, String spannerChangeStream, + String ignoreFieldsStr, TupleTag transformOut, TupleTag> transformDeadLetterOut) { this.spannerConfig = spannerConfig; this.spannerChangeStream = spannerChangeStream; this.transformOut = transformOut; this.transformDeadLetterOut = transformDeadLetterOut; + this.ignoreFields = new HashSet<>(); + for (String ignoreField : ignoreFieldsStr.split(",")) { + ignoreFields.add(ignoreField); + } } @Setup @@ -134,6 +143,11 @@ public void processElement(ProcessContext context) { try { TableRow tableRow = modJsonStringToTableRow(failsafeModJsonString.getPayload()); + for (String ignoreField : ignoreFields) { + if (tableRow.containsKey(ignoreField)) { + tableRow.remove(ignoreField); + } + } context.output(tableRow); } catch (Exception e) { context.output( @@ -249,6 +263,8 @@ public abstract static class FailsafeModJsonToTableRowOptions implements Seriali public abstract String getSpannerChangeStream(); + public abstract String getIgnoreFields(); + public abstract FailsafeElementCoder getCoder(); static Builder builder() { @@ -262,6 +278,8 @@ abstract static class Builder { abstract Builder setSpannerChangeStream(String spannerChangeStream); + abstract Builder setIgnoreFields(String ignoreFields); + abstract Builder setCoder(FailsafeElementCoder coder); abstract FailsafeModJsonToTableRowOptions build(); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java index 04287ff92b..690a181ec9 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java @@ -218,6 +218,7 @@ public void process( FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions.builder() .setSpannerConfig(spannerConfig) .setSpannerChangeStream(options.getSpannerChangeStream()) + .setIgnoreFields(options.getIgnoreFields()) .setCoder(FAILSAFE_ELEMENT_CODER) .build(); FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow failsafeModJsonToTableRow = @@ -298,7 +299,6 @@ private static DeadLetterQueueManager buildDlqManager( options.as(DataflowPipelineOptions.class).getTempLocation().endsWith("/") ? options.as(DataflowPipelineOptions.class).getTempLocation() : options.as(DataflowPipelineOptions.class).getTempLocation() + "/"; - String dlqDirectory = options.getDlqDirectory().isEmpty() ? tempLocation + "dlq/" : options.getDlqDirectory(); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java index e024ef4841..f1739e14a3 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java @@ -355,6 +355,7 @@ private static FailsafeModJsonToTableRow getFailsafeModJsonToTableRow( .setSpannerConfig(SPANNER_SERVER.getSpannerConfig(spannerDatabaseName)) .setSpannerChangeStream(TEST_SPANNER_CHANGE_STREAM) .setCoder(SpannerChangeStreamsToBigQuery.FAILSAFE_ELEMENT_CODER) + .setIgnoreFields("") .build(); return new FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow( failsafeModJsonToTableRowOptions); From c94282a89764d6d99c80e2bd363847a8b89e4785 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Wed, 27 Apr 2022 11:19:18 -0700 Subject: [PATCH 112/145] Fix a bug in Spanner change streams to BigQuery Dataflow template PiperOrigin-RevId: 444917262 --- .../schemautils/SpannerToBigQueryUtils.java | 55 ++++++++++--------- ...lsafeModJsonToTableRowTransformerTest.java | 36 ++++++------ .../SchemaUtilsTest.java | 36 ++++++------ .../TestUtils.java | 43 +++++++++++---- 4 files changed, 97 insertions(+), 73 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java index e3b1cc400a..40cb356afa 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java @@ -17,9 +17,6 @@ import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableRow; -import com.google.cloud.ByteArray; -import com.google.cloud.Date; -import com.google.cloud.Timestamp; import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.StandardSQLTypeName; import com.google.cloud.spanner.ResultSet; @@ -27,6 +24,8 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.TrackedSpannerColumn; import java.util.ArrayList; import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; import org.json.JSONArray; import org.json.JSONObject; @@ -143,37 +142,32 @@ private static Object getColumnValueFromResultSet( return null; } + // All the NULL columns in the array are filtered out since BigQuery doesn't allow NULL in the + // array. if (columnType.equals(Type.array(Type.bool()))) { - return resultSet.getBooleanList(columnName); + return removeNulls(resultSet.getBooleanList(columnName)); } else if (columnType.equals(Type.array(Type.bytes()))) { - List bytesList = resultSet.getBytesList(columnName); - List result = new ArrayList<>(); - for (ByteArray bytes : bytesList) { - result.add(bytes.toBase64()); - } - return result; + return removeNulls(resultSet.getBytesList(columnName)).stream() + .map(e -> e.toBase64()) + .collect(Collectors.toList()); } else if (columnType.equals(Type.array(Type.date()))) { - List result = new ArrayList<>(); - for (Date date : resultSet.getDateList(columnName)) { - result.add(date.toString()); - } - return result; + return removeNulls(resultSet.getDateList(columnName)).stream() + .map(e -> e.toString()) + .collect(Collectors.toList()); } else if (columnType.equals(Type.array(Type.float64()))) { - return resultSet.getDoubleList(columnName); + return removeNulls(resultSet.getDoubleList(columnName)); } else if (columnType.equals(Type.array(Type.int64()))) { - return resultSet.getLongList(columnName); + return removeNulls(resultSet.getLongList(columnName)); } else if (columnType.equals(Type.array(Type.json()))) { - return resultSet.getJsonList(columnName); + return removeNulls(resultSet.getJsonList(columnName)); } else if (columnType.equals(Type.array(Type.numeric()))) { - return resultSet.getBigDecimalList(columnName); + return removeNulls(resultSet.getBigDecimalList(columnName)); } else if (columnType.equals(Type.array(Type.string()))) { - return resultSet.getStringList(columnName); + return removeNulls(resultSet.getStringList(columnName)); } else if (columnType.equals(Type.array(Type.timestamp()))) { - List result = new ArrayList<>(); - for (Timestamp timestamp : resultSet.getTimestampList(columnName)) { - result.add(timestamp.toString()); - } - return result; + return removeNulls(resultSet.getTimestampList(columnName)).stream() + .map(e -> e.toString()) + .collect(Collectors.toList()); } else { Type.Code columnTypeCode = columnType.getCode(); switch (columnTypeCode) { @@ -202,6 +196,10 @@ private static Object getColumnValueFromResultSet( } } + private static List removeNulls(List list) { + return list.stream().filter(Objects::nonNull).collect(Collectors.toList()); + } + public static void addSpannerNonPkColumnsToTableRow( String newValuesJson, List spannerNonPkColumns, TableRow tableRow) { JSONObject newValuesJsonObject = new JSONObject(newValuesJson); @@ -223,8 +221,11 @@ public static void addSpannerNonPkColumnsToTableRow( || columnType.equals(Type.array(Type.timestamp()))) { JSONArray jsonArray = newValuesJsonObject.getJSONArray(columnName); List objects = new ArrayList<>(jsonArray.length()); - for (Object o : jsonArray) { - objects.add(o); + for (int i = 0; i < jsonArray.length(); i++) { + // BigQuery array doesn't allow NULL values. + if (!jsonArray.isNull(i)) { + objects.add(jsonArray.get(i)); + } } tableRow.set(columnName, objects); } else { diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java index f1739e14a3..7f1a6eaf71 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java @@ -17,56 +17,56 @@ import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_VAL; @@ -74,8 +74,8 @@ import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TEST_SPANNER_TABLE; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_RAW_VAL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_PK_COL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_RAW_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_VAL; @@ -376,15 +376,15 @@ private static Timestamp insertRow(String spannerDatabaseName) { .set(NUMERIC_PK_COL).to(NUMERIC_VAL) .set(STRING_PK_COL).to(STRING_VAL) .set(TIMESTAMP_PK_COL).to(TIMESTAMP_VAL) - .set(BOOLEAN_ARRAY_COL).to(BOOLEAN_ARRAY_VAL) - .set(BYTES_ARRAY_COL).to(BYTES_ARRAY_VAL) - .set(DATE_ARRAY_COL).to(DATE_ARRAY_VAL) - .set(FLOAT64_ARRAY_COL).to(FLOAT64_ARRAY_VAL) - .set(INT64_ARRAY_COL).to(INT64_ARRAY_VAL) - .set(NUMERIC_ARRAY_COL).to(NUMERIC_ARRAY_VAL) - .set(JSON_ARRAY_COL).to(JSON_ARRAY_VAL) - .set(STRING_ARRAY_COL).to(STRING_ARRAY_VAL) - .set(TIMESTAMP_ARRAY_COL).to(TIMESTAMP_ARRAY_VAL) + .set(BOOLEAN_ARRAY_COL).to(BOOLEAN_NULLABLE_ARRAY_VAL) + .set(BYTES_ARRAY_COL).to(BYTES_NULLABLE_ARRAY_VAL) + .set(DATE_ARRAY_COL).to(DATE_NULLABLE_ARRAY_VAL) + .set(FLOAT64_ARRAY_COL).to(FLOAT64_NULLABLE_ARRAY_VAL) + .set(INT64_ARRAY_COL).to(INT64_NULLABLE_ARRAY_VAL) + .set(NUMERIC_ARRAY_COL).to(NUMERIC_NULLABLE_ARRAY_VAL) + .set(JSON_ARRAY_COL).to(JSON_NULLABLE_ARRAY_VAL) + .set(STRING_ARRAY_COL).to(STRING_NULLABLE_ARRAY_VAL) + .set(TIMESTAMP_ARRAY_COL).to(TIMESTAMP_NULLABLE_ARRAY_VAL) .set(BOOLEAN_COL).to(BOOLEAN_VAL) .set(BYTES_COL).to(BYTES_VAL) .set(DATE_COL).to(DATE_VAL) diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java index 61d6c6875b..4377cda8d4 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java @@ -16,40 +16,40 @@ package com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BOOLEAN_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.BYTES_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.DATE_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.FLOAT64_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.INT64_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.JSON_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.NUMERIC_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.STRING_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_COL; -import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_COL; +import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_NULLABLE_ARRAY_VAL; import static com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.TestUtils.TIMESTAMP_VAL; import static com.google.common.truth.Truth.assertThat; import static org.mockito.Mockito.when; @@ -248,15 +248,15 @@ public void testSpannerSnapshotRowToBigQueryTableRow() { .set(NUMERIC_COL).to(NUMERIC_VAL) .set(STRING_COL).to(STRING_VAL) .set(TIMESTAMP_COL).to(TIMESTAMP_VAL) - .set(BOOLEAN_ARRAY_COL).to(BOOLEAN_ARRAY_VAL) - .set(BYTES_ARRAY_COL).to(BYTES_ARRAY_VAL) - .set(DATE_ARRAY_COL).to(DATE_ARRAY_VAL) - .set(FLOAT64_ARRAY_COL).to(FLOAT64_ARRAY_VAL) - .set(INT64_ARRAY_COL).to(INT64_ARRAY_VAL) - .set(JSON_ARRAY_COL).to(JSON_ARRAY_VAL) - .set(NUMERIC_ARRAY_COL).to(NUMERIC_ARRAY_VAL) - .set(STRING_ARRAY_COL).to(STRING_ARRAY_VAL) - .set(TIMESTAMP_ARRAY_COL).to(TIMESTAMP_ARRAY_VAL) + .set(BOOLEAN_ARRAY_COL).to(BOOLEAN_NULLABLE_ARRAY_VAL) + .set(BYTES_ARRAY_COL).to(BYTES_NULLABLE_ARRAY_VAL) + .set(DATE_ARRAY_COL).to(DATE_NULLABLE_ARRAY_VAL) + .set(FLOAT64_ARRAY_COL).to(FLOAT64_NULLABLE_ARRAY_VAL) + .set(INT64_ARRAY_COL).to(INT64_NULLABLE_ARRAY_VAL) + .set(JSON_ARRAY_COL).to(JSON_NULLABLE_ARRAY_VAL) + .set(NUMERIC_ARRAY_COL).to(NUMERIC_NULLABLE_ARRAY_VAL) + .set(STRING_ARRAY_COL).to(STRING_NULLABLE_ARRAY_VAL) + .set(TIMESTAMP_ARRAY_COL).to(TIMESTAMP_NULLABLE_ARRAY_VAL) .build())); // spotless:on SpannerToBigQueryUtils.spannerSnapshotRowToBigQueryTableRow( diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java index c4c14a9fdf..d8fd54f750 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/TestUtils.java @@ -21,6 +21,7 @@ import com.google.cloud.spanner.Value; import com.google.cloud.teleport.v2.spanner.SpannerServerResource; import java.math.BigDecimal; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import org.apache.commons.lang3.RandomStringUtils; @@ -110,18 +111,40 @@ class TestUtils { Timestamp.ofTimeSecondsAndNanos(1646637853L, 572000000), Timestamp.ofTimeSecondsAndNanos(1646657853L, 772000000)); - static final Value BOOLEAN_ARRAY_VAL = Value.boolArray(BOOLEAN_ARRAY_RAW_VAL); - static final Value BYTES_ARRAY_VAL = + static final List BOOLEAN_NULLABLE_ARRAY_RAW_VAL = addNull(BOOLEAN_ARRAY_RAW_VAL); + static final List BYTES_NULLABLE_ARRAY_RAW_VAL = addNull(BYTES_ARRAY_RAW_VAL); + static final List DATE_NULLABLE_ARRAY_RAW_VAL = addNull(DATE_ARRAY_RAW_VAL); + static final List FLOAT64_NULLABLE_ARRAY_RAW_VAL = addNull(FLOAT64_ARRAY_RAW_VAL); + static final List INT64_NULLABLE_ARRAY_RAW_VAL = addNull(INT64_ARRAY_RAW_VAL); + static final List JSON_NULLABLE_ARRAY_RAW_VAL = addNull(JSON_ARRAY_RAW_VAL); + static final List NUMERIC_NULLABLE_ARRAY_RAW_VAL = addNull(NUMERIC_ARRAY_RAW_VAL); + static final List STRING_NULLABLE_ARRAY_RAW_VAL = addNull(STRING_ARRAY_RAW_VAL); + static final List TIMESTAMP_NULLABLE_ARRAY_RAW_VAL = addNull(TIMESTAMP_ARRAY_RAW_VAL); + + static final Value BOOLEAN_NULLABLE_ARRAY_VAL = Value.boolArray(BOOLEAN_ARRAY_RAW_VAL); + static final Value BYTES_NULLABLE_ARRAY_VAL = Value.bytesArray( Arrays.asList( - ByteArray.copyFrom("123"), ByteArray.copyFrom("456"), ByteArray.copyFrom("789"))); - static final Value DATE_ARRAY_VAL = Value.dateArray(DATE_ARRAY_RAW_VAL); - static final Value FLOAT64_ARRAY_VAL = Value.float64Array(FLOAT64_ARRAY_RAW_VAL); - static final Value INT64_ARRAY_VAL = Value.int64Array(INT64_ARRAY_RAW_VAL); - static final Value JSON_ARRAY_VAL = Value.jsonArray(JSON_ARRAY_RAW_VAL); - static final Value NUMERIC_ARRAY_VAL = Value.numericArray(NUMERIC_ARRAY_RAW_VAL); - static final Value STRING_ARRAY_VAL = Value.stringArray(STRING_ARRAY_RAW_VAL); - static final Value TIMESTAMP_ARRAY_VAL = Value.timestampArray(TIMESTAMP_ARRAY_RAW_VAL); + ByteArray.copyFrom("123"), + ByteArray.copyFrom("456"), + ByteArray.copyFrom("789"), + null)); + static final Value DATE_NULLABLE_ARRAY_VAL = Value.dateArray(DATE_NULLABLE_ARRAY_RAW_VAL); + static final Value FLOAT64_NULLABLE_ARRAY_VAL = + Value.float64Array(FLOAT64_NULLABLE_ARRAY_RAW_VAL); + static final Value INT64_NULLABLE_ARRAY_VAL = Value.int64Array(INT64_NULLABLE_ARRAY_RAW_VAL); + static final Value JSON_NULLABLE_ARRAY_VAL = Value.jsonArray(JSON_NULLABLE_ARRAY_RAW_VAL); + static final Value NUMERIC_NULLABLE_ARRAY_VAL = + Value.numericArray(NUMERIC_NULLABLE_ARRAY_RAW_VAL); + static final Value STRING_NULLABLE_ARRAY_VAL = Value.stringArray(STRING_NULLABLE_ARRAY_RAW_VAL); + static final Value TIMESTAMP_NULLABLE_ARRAY_VAL = + Value.timestampArray(TIMESTAMP_NULLABLE_ARRAY_RAW_VAL); + + private static List addNull(List list) { + List result = new ArrayList<>(list); + result.add(null); + return result; + } private static String generateSpannerDatabaseName() { return TEST_SPANNER_DATABASE_PREFIX From 1aaab5d1066cc997a303497efccca7026dd40a89 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Sun, 1 May 2022 16:47:21 -0700 Subject: [PATCH 113/145] Add better error message if JavaScript engine is null. PiperOrigin-RevId: 445813901 --- .../common/JavascriptTextTransformer.java | 14 +++++++++++++- .../v2/transforms/JavascriptTextTransformer.java | 11 +++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/JavascriptTextTransformer.java b/src/main/java/com/google/cloud/teleport/templates/common/JavascriptTextTransformer.java index d10980c072..292832e086 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/JavascriptTextTransformer.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/JavascriptTextTransformer.java @@ -27,12 +27,14 @@ import java.io.UncheckedIOException; import java.nio.channels.Channels; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.script.Invocable; import javax.script.ScriptEngine; +import javax.script.ScriptEngineFactory; import javax.script.ScriptEngineManager; import javax.script.ScriptException; import jdk.nashorn.api.scripting.ScriptObjectMirror; @@ -140,6 +142,15 @@ private static Invocable newInvocable(Collection scripts) throws ScriptE ScriptEngineManager manager = new ScriptEngineManager(); ScriptEngine engine = manager.getEngineByName("JavaScript"); + if (engine == null) { + List availableEngines = new ArrayList<>(); + for (ScriptEngineFactory factory : manager.getEngineFactories()) { + availableEngines.add(factory.getEngineName() + " " + factory.getEngineVersion()); + } + throw new RuntimeException( + String.format("JavaScript engine not available. Found engines: %s.", availableEngines)); + } + for (String script : scripts) { engine.eval(script); } @@ -337,7 +348,8 @@ public void processElement(ProcessContext context) { } catch (ScriptException | IOException | NoSuchMethodException e) { if (loggingEnabled) { LOG.warn( - "Exception occurred while applying UDF '{}' from file path '{}' due to '{}'", + "Exception occurred while applying UDF '{}' from file path '{}' due" + + " to '{}'", functionName().get(), fileSystemPath().get(), e.getMessage()); diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/JavascriptTextTransformer.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/JavascriptTextTransformer.java index 834b8e3928..2120a91ed1 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/JavascriptTextTransformer.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/JavascriptTextTransformer.java @@ -24,12 +24,14 @@ import java.io.UncheckedIOException; import java.nio.channels.Channels; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.script.Invocable; import javax.script.ScriptEngine; +import javax.script.ScriptEngineFactory; import javax.script.ScriptEngineManager; import javax.script.ScriptException; import jdk.nashorn.api.scripting.ScriptObjectMirror; @@ -136,6 +138,15 @@ private static Invocable newInvocable(Collection scripts) throws ScriptE ScriptEngineManager manager = new ScriptEngineManager(); ScriptEngine engine = manager.getEngineByName("JavaScript"); + if (engine == null) { + List availableEngines = new ArrayList<>(); + for (ScriptEngineFactory factory : manager.getEngineFactories()) { + availableEngines.add(factory.getEngineName() + " " + factory.getEngineVersion()); + } + throw new RuntimeException( + String.format("JavaScript engine not available. Found engines: %s.", availableEngines)); + } + for (String script : scripts) { engine.eval(script); } From 43a991a7b6c628e85e1e7499febcbb21a9e541bc Mon Sep 17 00:00:00 2001 From: pabloem Date: Tue, 3 May 2022 16:04:00 -0700 Subject: [PATCH 114/145] Adding Pane Info to Spanner template to unblock migration. PiperOrigin-RevId: 446312743 --- .../google/cloud/teleport/v2/templates/DataStreamToSpanner.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/v2/datastream-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSpanner.java b/v2/datastream-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSpanner.java index 40a18c5e81..e9c7a02d08 100644 --- a/v2/datastream-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSpanner.java +++ b/v2/datastream-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/DataStreamToSpanner.java @@ -356,6 +356,7 @@ public static PipelineResult run(Options options) { DLQWriteTransform.WriteDLQ.newBuilder() .withDlqDirectory(dlqManager.getRetryDlqDirectoryWithDateTime()) .withTmpDirectory(dlqManager.getRetryDlqDirectory() + "tmp/") + .setIncludePaneInfo(true) .build()); PCollection> dlqErrorRecords = @@ -379,6 +380,7 @@ public static PipelineResult run(Options options) { DLQWriteTransform.WriteDLQ.newBuilder() .withDlqDirectory(dlqManager.getSevereDlqDirectoryWithDateTime()) .withTmpDirectory(dlqManager.getSevereDlqDirectory() + "tmp/") + .setIncludePaneInfo(true) .build()); // Execute the pipeline and return the result. From a0c18a015ab2f2450ba171b007b604547b38bcd9 Mon Sep 17 00:00:00 2001 From: anikin Date: Tue, 3 May 2022 17:34:51 -0700 Subject: [PATCH 115/145] Spanner templates: Explicitly set spannerProjectId to the default project if not provided as a template parameter, as a workaround for a bug in Beam 2.38. PiperOrigin-RevId: 446330531 --- .../cloud/teleport/spanner/ExportPipeline.java | 13 ++++++++++++- .../cloud/teleport/spanner/ImportPipeline.java | 13 ++++++++++++- .../cloud/teleport/spanner/TextImportPipeline.java | 13 ++++++++++++- 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java index 260e426bd4..d79dbff514 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportPipeline.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner; import com.google.cloud.spanner.Options.RpcPriority; +import com.google.cloud.spanner.SpannerOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -25,6 +26,8 @@ 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; /** Dataflow template that exports a Cloud Spanner database to Avro files in GCS. */ public class ExportPipeline { @@ -129,7 +132,15 @@ public static void main(String[] args) { SpannerConfig spannerConfig = SpannerConfig.create() - .withProjectId(options.getSpannerProjectId()) + // 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) input -> + input != null ? input : SpannerOptions.getDefaultProjectId())) .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) .withDatabaseId(options.getDatabaseId()) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java index 2b539fad00..f38fa17e4a 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner; import com.google.cloud.spanner.Options.RpcPriority; +import com.google.cloud.spanner.SpannerOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -25,6 +26,8 @@ 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; /** Avro to Cloud Spanner Import pipeline. */ public class ImportPipeline { @@ -113,7 +116,15 @@ public static void main(String[] args) { SpannerConfig spannerConfig = SpannerConfig.create() - .withProjectId(options.getSpannerProjectId()) + // 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) input -> + input != null ? input : SpannerOptions.getDefaultProjectId())) .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) .withDatabaseId(options.getDatabaseId()) diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java index cb5b2bf571..c6d9d15996 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextImportPipeline.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner; import com.google.cloud.spanner.Options.RpcPriority; +import com.google.cloud.spanner.SpannerOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -25,6 +26,8 @@ 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 @@ -184,7 +187,15 @@ public static void main(String[] args) { SpannerConfig spannerConfig = SpannerConfig.create() - .withProjectId(options.getSpannerProjectId()) + // 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) input -> + input != null ? input : SpannerOptions.getDefaultProjectId())) .withHost(options.getSpannerHost()) .withInstanceId(options.getInstanceId()) .withDatabaseId(options.getDatabaseId()) From d29030921c8196e641e80370656a47a2c10c00fb Mon Sep 17 00:00:00 2001 From: anikin Date: Wed, 4 May 2022 06:49:50 -0700 Subject: [PATCH 116/145] Upgrade to Beam 2.38 PiperOrigin-RevId: 446442639 --- .../it/dataflow/DataflowOperatorTest.java | 4 +- pom.xml | 12 ++- .../io/gcp/spanner/ExposedSpannerConfig.java | 81 +++++++++++++++++++ v2/googlecloud-to-googlecloud/pom.xml | 6 ++ v2/pom.xml | 14 +++- 5 files changed, 111 insertions(+), 6 deletions(-) diff --git a/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java index 003247017a..f03a561832 100644 --- a/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java +++ b/it/src/test/java/com/google/cloud/teleport/it/dataflow/DataflowOperatorTest.java @@ -54,8 +54,8 @@ public final class DataflowOperatorTest { private static final String PROJECT = "test-project"; private static final String REGION = "us-east1"; private static final String JOB_ID = "test-job-id"; - private static final Duration CHECK_AFTER = Duration.ofMillis(10); - private static final Duration TIMEOUT_AFTER = Duration.ofMillis(100); + private static final Duration CHECK_AFTER = Duration.ofMillis(100); + private static final Duration TIMEOUT_AFTER = Duration.ofMillis(1000); private static final Config DEFAULT_CONFIG = Config.builder() diff --git a/pom.xml b/pom.xml index 6cca8e6d0f..992a340cf0 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ 1.0-rc6 1.8.2 8.7 - 2.37.0 + 2.38.0 1.3 2.1 1.8 @@ -89,7 +89,7 @@ 1.40.0 0.87.0 2.12.1 - 24.2.0 + 24.4.0 @@ -107,7 +107,13 @@ ${cloud-libraries-bom.version} pom import - + + + com.google.http-client + google-http-client-bom + 1.41.2 + pom + diff --git a/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java b/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java index fce969fd2c..24a5af1c15 100644 --- a/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java +++ b/v2/datastream-to-spanner/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ExposedSpannerConfig.java @@ -15,6 +15,8 @@ */ package org.apache.beam.sdk.io.gcp.spanner; +import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.StatusCode.Code; import com.google.cloud.ServiceFactory; import com.google.cloud.spanner.Options.RpcPriority; import com.google.cloud.spanner.Spanner; @@ -22,6 +24,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet; import org.joda.time.Duration; /** Exposed spanner config. */ @@ -43,10 +46,18 @@ public class ExposedSpannerConfig extends SpannerConfig { private final ValueProvider emulatorHost; + private final ValueProvider isLocalChannelProvider; + private final ValueProvider commitDeadline; private final ValueProvider maxCumulativeBackoff; + private final RetrySettings executeStreamingSqlRetrySettings; + + private final RetrySettings commitRetrySettings; + + private final ImmutableSet retryableCodes; + private final ValueProvider rpcPriority; private final ServiceFactory serviceFactory; @@ -57,8 +68,12 @@ private ExposedSpannerConfig( @Nullable ValueProvider databaseId, @Nullable ValueProvider host, @Nullable ValueProvider emulatorHost, + @Nullable ValueProvider isLocalChannelProvider, @Nullable ValueProvider commitDeadline, @Nullable ValueProvider maxCumulativeBackoff, + @Nullable RetrySettings executeStreamingSqlRetrySettings, + @Nullable RetrySettings commitRetrySettings, + @Nullable ImmutableSet retryableCodes, @Nullable ValueProvider rpcPriority, @Nullable ServiceFactory serviceFactory) { this.projectId = projectId; @@ -66,8 +81,12 @@ private ExposedSpannerConfig( this.databaseId = databaseId; this.host = host; this.emulatorHost = emulatorHost; + this.isLocalChannelProvider = isLocalChannelProvider; this.commitDeadline = commitDeadline; this.maxCumulativeBackoff = maxCumulativeBackoff; + this.executeStreamingSqlRetrySettings = executeStreamingSqlRetrySettings; + this.commitRetrySettings = commitRetrySettings; + this.retryableCodes = retryableCodes; this.rpcPriority = rpcPriority; this.serviceFactory = serviceFactory; } @@ -102,6 +121,12 @@ public ValueProvider getEmulatorHost() { return emulatorHost; } + @Nullable + @Override + public ValueProvider getIsLocalChannelProvider() { + return isLocalChannelProvider; + } + @Nullable @Override public ValueProvider getCommitDeadline() { @@ -114,6 +139,24 @@ public ValueProvider getMaxCumulativeBackoff() { return maxCumulativeBackoff; } + @Nullable + @Override + public RetrySettings getExecuteStreamingSqlRetrySettings() { + return executeStreamingSqlRetrySettings; + } + + @Nullable + @Override + public RetrySettings getCommitRetrySettings() { + return commitRetrySettings; + } + + @Nullable + @Override + public ImmutableSet getRetryableCodes() { + return retryableCodes; + } + @Nullable @Override public ValueProvider getRpcPriority() { @@ -235,8 +278,12 @@ static final class Builder extends SpannerConfig.Builder { private ValueProvider databaseId; private ValueProvider host; private ValueProvider emulatorHost; + private ValueProvider isLocalChannelProvider; private ValueProvider commitDeadline; private ValueProvider maxCumulativeBackoff; + private RetrySettings executeStreamingSqlRetrySettings; + private RetrySettings commitRetrySettings; + private ImmutableSet retryableCodes; private ValueProvider rpcPriority; private ServiceFactory serviceFactory; @@ -248,8 +295,12 @@ private Builder(SpannerConfig source) { this.databaseId = source.getDatabaseId(); this.host = source.getHost(); this.emulatorHost = source.getEmulatorHost(); + this.isLocalChannelProvider = source.getIsLocalChannelProvider(); this.commitDeadline = source.getCommitDeadline(); this.maxCumulativeBackoff = source.getMaxCumulativeBackoff(); + this.executeStreamingSqlRetrySettings = source.getExecuteStreamingSqlRetrySettings(); + this.commitRetrySettings = source.getCommitRetrySettings(); + this.retryableCodes = source.getRetryableCodes(); this.rpcPriority = source.getRpcPriority(); this.serviceFactory = source.getServiceFactory(); } @@ -278,6 +329,13 @@ ExposedSpannerConfig.Builder setHost(ValueProvider host) { return this; } + @Override + ExposedSpannerConfig.Builder setIsLocalChannelProvider( + ValueProvider isLocalChannelProvider) { + this.isLocalChannelProvider = isLocalChannelProvider; + return this; + } + @Override ExposedSpannerConfig.Builder setEmulatorHost(ValueProvider emulatorHost) { this.emulatorHost = emulatorHost; @@ -296,6 +354,25 @@ SpannerConfig.Builder setMaxCumulativeBackoff(ValueProvider maxCumulat return this; } + @Override + SpannerConfig.Builder setExecuteStreamingSqlRetrySettings( + RetrySettings executeStreamingSqlRetrySettings) { + this.executeStreamingSqlRetrySettings = executeStreamingSqlRetrySettings; + return this; + } + + @Override + SpannerConfig.Builder setCommitRetrySettings(RetrySettings commitRetrySettings) { + this.commitRetrySettings = commitRetrySettings; + return this; + } + + @Override + SpannerConfig.Builder setRetryableCodes(ImmutableSet retryableCodes) { + this.retryableCodes = retryableCodes; + return this; + } + @Override SpannerConfig.Builder setRpcPriority(ValueProvider rpcPriority) { this.rpcPriority = rpcPriority; @@ -317,8 +394,12 @@ public ExposedSpannerConfig build() { this.databaseId, this.host, this.emulatorHost, + this.isLocalChannelProvider, this.commitDeadline, this.maxCumulativeBackoff, + this.executeStreamingSqlRetrySettings, + this.commitRetrySettings, + this.retryableCodes, this.rpcPriority, this.serviceFactory); } diff --git a/v2/googlecloud-to-googlecloud/pom.xml b/v2/googlecloud-to-googlecloud/pom.xml index add2736f93..18d90bb187 100644 --- a/v2/googlecloud-to-googlecloud/pom.xml +++ b/v2/googlecloud-to-googlecloud/pom.xml @@ -64,6 +64,12 @@ org.apache.beam beam-sdks-java-io-google-cloud-platform + + + io.grpc + grpc-netty-shaded + + org.apache.beam diff --git a/v2/pom.xml b/v2/pom.xml index 4954518c57..197b1b5c84 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -29,7 +29,7 @@ 1.7.4 1.0-rc6 8.7 - 2.37.0 + 2.38.0 2.1 2.10.1 30.1-jre @@ -98,6 +98,18 @@ junit junit + + io.grpc + grpc-netty-shaded + + + io.grpc + grpc-core + + + io.grpc + grpc-api + From 0f2353979e797b811727953d3149f50531d87048 Mon Sep 17 00:00:00 2001 From: anikin Date: Thu, 5 May 2022 06:45:52 -0700 Subject: [PATCH 117/145] Switch GitHub actions to Java 11 by default. Some dependencies like google-java-format are now compiled with java11 too, breaking the build and unit test actions that try to build the project using java8. PiperOrigin-RevId: 446704077 --- .github/actions/setup-env/action.yml | 2 +- .github/workflows/java-pr.yml | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/.github/actions/setup-env/action.yml b/.github/actions/setup-env/action.yml index 6fdd8cfbe5..6a33f524fe 100644 --- a/.github/actions/setup-env/action.yml +++ b/.github/actions/setup-env/action.yml @@ -35,7 +35,7 @@ inputs: type: string description: 'The version of Java to install' required: false - default: '8' + default: '11' go-version: type: string description: 'The version of Go to install' diff --git a/.github/workflows/java-pr.yml b/.github/workflows/java-pr.yml index 084a0911c6..9458966543 100644 --- a/.github/workflows/java-pr.yml +++ b/.github/workflows/java-pr.yml @@ -41,8 +41,6 @@ jobs: - name: Setup Environment id: setup-env uses: ./.github/actions/setup-env - with: - java-version: '11' # Spotless won't work on version 8 - name: Run Spotless run: ./cicd/run-spotless --changed-files="${{ steps.setup-env.outputs.changed-files }}" java_build: From b761b565a099c5b0a10a8e217bc1eee50c2de15e Mon Sep 17 00:00:00 2001 From: Cherepushko Date: Thu, 24 Mar 2022 14:38:23 +0200 Subject: [PATCH 118/145] fixed compatibility issue --- .../elasticsearch/utils/ElasticsearchIO.java | 61 +++++++++++++------ .../transforms/ProcessFailsafePubSubFn.java | 39 +++++++++++- 2 files changed, 80 insertions(+), 20 deletions(-) diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java index eea718c32d..339d064dac 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java @@ -207,12 +207,16 @@ static void checkForErrors(HttpEntity responseEntity, int backendVersion, boolea if (partialUpdate) { errorRootName = "update"; } else { - if (backendVersion == 2) { - errorRootName = "create"; - } else if (backendVersion >= 5) { - errorRootName = "index"; + // look for error object key dynamically + for (String name : new String[] {"create", "index"}) { + JsonNode root = item.path(name); + if (root != null && !root.isMissingNode()) { + errorRootName = name; + break; + } } } + JsonNode errorRoot = item.path(errorRootName); JsonNode error = errorRoot.get("error"); if (error != null) { @@ -752,10 +756,16 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { return estimatedByteSize; } - String endPoint = - String.format( - "/%s/%s/_count", - connectionConfiguration.getIndex(), connectionConfiguration.getType()); + String endPoint; + if (backendVersion < 6) { + endPoint = + String.format( + "/%s/%s/_count", + connectionConfiguration.getIndex(), connectionConfiguration.getType()); + } else { + endPoint = String.format("/%s/_count", connectionConfiguration.getIndex()); + } + try (RestClient restClient = connectionConfiguration.createClient()) { long count = queryCount(restClient, endPoint, query); LOG.debug("estimate source byte size: query document count " + count); @@ -862,11 +872,17 @@ public boolean start() throws IOException { String.format("\"slice\": {\"id\": %s,\"max\": %s}", source.sliceId, source.numSlices); query = query.replaceFirst("\\{", "{" + sliceQuery + ","); } - String endPoint = + String endPoint; + if (source.backendVersion < 6) { + endPoint = String.format( - "/%s/%s/_search", - source.spec.getConnectionConfiguration().getIndex(), - source.spec.getConnectionConfiguration().getType()); + "/%s/%s/_search", + source.spec.getConnectionConfiguration().getIndex(), + source.spec.getConnectionConfiguration().getType()); + } else { + endPoint = String.format("/%s/_search", source.spec.getConnectionConfiguration().getIndex()); + } + Map params = new HashMap<>(); params.put("scroll", source.spec.getScrollKeepalive()); if (source.backendVersion == 2) { @@ -1342,7 +1358,7 @@ public void setup() throws IOException { // configure a custom serializer for metadata to be able to change serialization based // on ES version SimpleModule module = new SimpleModule(); - module.addSerializer(DocumentMetadata.class, new DocumentMetadataSerializer()); + module.addSerializer(DocumentMetadata.class, new DocumentMetadataSerializer((backendVersion >= 7))); OBJECT_MAPPER.registerModule(module); } @@ -1353,9 +1369,11 @@ public void startBundle(StartBundleContext context) { } private class DocumentMetadataSerializer extends StdSerializer { + private boolean excludeType = false; - private DocumentMetadataSerializer() { + private DocumentMetadataSerializer(boolean excludeType) { super(DocumentMetadata.class); + this.excludeType = excludeType; } @Override @@ -1366,7 +1384,7 @@ public void serialize( if (value.index != null) { gen.writeStringField("_index", value.index); } - if (value.type != null) { + if (value.type != null && !this.excludeType) { gen.writeStringField("_type", value.type); } if (value.id != null) { @@ -1464,11 +1482,16 @@ private void flushBatch() throws IOException, InterruptedException { // Elasticsearch will default to the index/type provided here if none are set in the // document meta (i.e. using ElasticsearchIO$Write#withIndexFn and // ElasticsearchIO$Write#withTypeFn options) - String endPoint = + String endPoint; + if (backendVersion < 6) { + endPoint = String.format( - "/%s/%s/_bulk", - spec.getConnectionConfiguration().getIndex(), - spec.getConnectionConfiguration().getType()); + "/%s/%s/_bulk", + spec.getConnectionConfiguration().getIndex(), + spec.getConnectionConfiguration().getType()); + } else { + endPoint = String.format("/%s/_bulk", spec.getConnectionConfiguration().getIndex()); + } HttpEntity requestBody = new NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON); Request request = new Request("POST", endPoint); diff --git a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java index 6247dc1043..766e7f9b51 100644 --- a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java +++ b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java @@ -18,6 +18,7 @@ import com.google.cloud.teleport.v2.elasticsearch.templates.PubSubToElasticsearch; import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.gson.Gson; +import com.google.gson.JsonElement; import com.google.gson.JsonObject; import com.google.gson.JsonSyntaxException; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; @@ -26,6 +27,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + /** * The {@link ProcessFailsafePubSubFn} class processes a {@link FailsafeElement} containing an * {@link PubsubMessage} and a String of the message's payload {@link PubsubMessage#getPayload()} @@ -46,6 +52,37 @@ public class ProcessFailsafePubSubFn private static final Counter failedCounter = Metrics.counter(PubSubMessageToJsonDocument.class, "failed-messages-processed"); + // Elasticsearch cannot accept fields containing only dots + public static JsonObject patchJsonFields(JsonObject jsonObject) { + Set> entrySet = jsonObject.entrySet(); + List keysToFix = new ArrayList<>(); + for (Map.Entry entry : entrySet) { + String key = entry.getKey(); + if (key.equals(".") || key.equals("..")) { + keysToFix.add(key); + } + if (entry.getValue().isJsonObject()) { + patchJsonFields(entry.getValue().getAsJsonObject()); + } + if (entry.getValue().isJsonArray()) { + for (JsonElement ae : entry.getValue().getAsJsonArray()) { + if (ae.isJsonObject()) { + patchJsonFields(ae.getAsJsonObject()); + } + } + } + } + for (String key : keysToFix) { + if (key.equals(".")) { + jsonObject.add("@", jsonObject.get(key)); + } else if (key.equals("..")) { + jsonObject.add("@@", jsonObject.get(key)); + } + jsonObject.remove(key); + } + return jsonObject; + } + @ProcessElement public void processElement(ProcessContext context) { PubsubMessage pubsubMessage = context.element().getOriginalPayload(); @@ -54,7 +91,7 @@ public void processElement(ProcessContext context) { try { if (pubsubMessage.getPayload().length > 0) { - messageObject = gson.fromJson(new String(pubsubMessage.getPayload()), JsonObject.class); + messageObject = patchJsonFields(gson.fromJson(new String(pubsubMessage.getPayload()), JsonObject.class)); } // If message attributes are present they will be serialized along with the message payload From 0a440a3a29423b146997c923db1e6441d81cabe2 Mon Sep 17 00:00:00 2001 From: Cherepushko Date: Fri, 25 Mar 2022 15:22:03 +0200 Subject: [PATCH 119/145] spotless --- .../elasticsearch/utils/ElasticsearchIO.java | 28 ++++++++++--------- .../transforms/ProcessFailsafePubSubFn.java | 13 +++++---- 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java index 339d064dac..e6445ae6ce 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java @@ -759,9 +759,9 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { String endPoint; if (backendVersion < 6) { endPoint = - String.format( - "/%s/%s/_count", - connectionConfiguration.getIndex(), connectionConfiguration.getType()); + String.format( + "/%s/%s/_count", + connectionConfiguration.getIndex(), connectionConfiguration.getType()); } else { endPoint = String.format("/%s/_count", connectionConfiguration.getIndex()); } @@ -875,12 +875,13 @@ public boolean start() throws IOException { String endPoint; if (source.backendVersion < 6) { endPoint = - String.format( - "/%s/%s/_search", - source.spec.getConnectionConfiguration().getIndex(), - source.spec.getConnectionConfiguration().getType()); + String.format( + "/%s/%s/_search", + source.spec.getConnectionConfiguration().getIndex(), + source.spec.getConnectionConfiguration().getType()); } else { - endPoint = String.format("/%s/_search", source.spec.getConnectionConfiguration().getIndex()); + endPoint = + String.format("/%s/_search", source.spec.getConnectionConfiguration().getIndex()); } Map params = new HashMap<>(); @@ -1358,7 +1359,8 @@ public void setup() throws IOException { // configure a custom serializer for metadata to be able to change serialization based // on ES version SimpleModule module = new SimpleModule(); - module.addSerializer(DocumentMetadata.class, new DocumentMetadataSerializer((backendVersion >= 7))); + module.addSerializer( + DocumentMetadata.class, new DocumentMetadataSerializer((backendVersion >= 7))); OBJECT_MAPPER.registerModule(module); } @@ -1485,10 +1487,10 @@ private void flushBatch() throws IOException, InterruptedException { String endPoint; if (backendVersion < 6) { endPoint = - String.format( - "/%s/%s/_bulk", - spec.getConnectionConfiguration().getIndex(), - spec.getConnectionConfiguration().getType()); + String.format( + "/%s/%s/_bulk", + spec.getConnectionConfiguration().getIndex(), + spec.getConnectionConfiguration().getType()); } else { endPoint = String.format("/%s/_bulk", spec.getConnectionConfiguration().getIndex()); } diff --git a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java index 766e7f9b51..adebac553e 100644 --- a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java +++ b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java @@ -21,17 +21,16 @@ import com.google.gson.JsonElement; import com.google.gson.JsonObject; import com.google.gson.JsonSyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * The {@link ProcessFailsafePubSubFn} class processes a {@link FailsafeElement} containing an * {@link PubsubMessage} and a String of the message's payload {@link PubsubMessage#getPayload()} @@ -91,7 +90,9 @@ public void processElement(ProcessContext context) { try { if (pubsubMessage.getPayload().length > 0) { - messageObject = patchJsonFields(gson.fromJson(new String(pubsubMessage.getPayload()), JsonObject.class)); + messageObject = + patchJsonFields( + gson.fromJson(new String(pubsubMessage.getPayload()), JsonObject.class)); } // If message attributes are present they will be serialized along with the message payload From adf7e34afaf98b110dd8ec8521a02b9f994e3fe1 Mon Sep 17 00:00:00 2001 From: Cherepushko Date: Fri, 25 Mar 2022 16:42:25 +0200 Subject: [PATCH 120/145] fixed backend version check --- .../teleport/v2/elasticsearch/utils/ElasticsearchIO.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java index e6445ae6ce..9dadd10260 100644 --- a/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java +++ b/v2/elasticsearch-common/src/main/java/com/google/cloud/teleport/v2/elasticsearch/utils/ElasticsearchIO.java @@ -757,7 +757,7 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { } String endPoint; - if (backendVersion < 6) { + if (backendVersion < 7) { endPoint = String.format( "/%s/%s/_count", @@ -873,7 +873,7 @@ public boolean start() throws IOException { query = query.replaceFirst("\\{", "{" + sliceQuery + ","); } String endPoint; - if (source.backendVersion < 6) { + if (source.backendVersion < 7) { endPoint = String.format( "/%s/%s/_search", @@ -1485,7 +1485,7 @@ private void flushBatch() throws IOException, InterruptedException { // document meta (i.e. using ElasticsearchIO$Write#withIndexFn and // ElasticsearchIO$Write#withTypeFn options) String endPoint; - if (backendVersion < 6) { + if (backendVersion < 7) { endPoint = String.format( "/%s/%s/_bulk", From 0ee982f8f7652626b3230928907b80c7d2abbb46 Mon Sep 17 00:00:00 2001 From: Cherepushko Date: Tue, 26 Apr 2022 01:59:13 +0300 Subject: [PATCH 121/145] moved dot fix to separate branch --- .../transforms/ProcessFailsafePubSubFn.java | 40 +------------------ 1 file changed, 1 insertion(+), 39 deletions(-) diff --git a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java index adebac553e..6247dc1043 100644 --- a/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java +++ b/v2/googlecloud-to-elasticsearch/src/main/java/com/google/cloud/teleport/v2/elasticsearch/transforms/ProcessFailsafePubSubFn.java @@ -18,13 +18,8 @@ import com.google.cloud.teleport.v2.elasticsearch.templates.PubSubToElasticsearch; import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.gson.Gson; -import com.google.gson.JsonElement; import com.google.gson.JsonObject; import com.google.gson.JsonSyntaxException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -51,37 +46,6 @@ public class ProcessFailsafePubSubFn private static final Counter failedCounter = Metrics.counter(PubSubMessageToJsonDocument.class, "failed-messages-processed"); - // Elasticsearch cannot accept fields containing only dots - public static JsonObject patchJsonFields(JsonObject jsonObject) { - Set> entrySet = jsonObject.entrySet(); - List keysToFix = new ArrayList<>(); - for (Map.Entry entry : entrySet) { - String key = entry.getKey(); - if (key.equals(".") || key.equals("..")) { - keysToFix.add(key); - } - if (entry.getValue().isJsonObject()) { - patchJsonFields(entry.getValue().getAsJsonObject()); - } - if (entry.getValue().isJsonArray()) { - for (JsonElement ae : entry.getValue().getAsJsonArray()) { - if (ae.isJsonObject()) { - patchJsonFields(ae.getAsJsonObject()); - } - } - } - } - for (String key : keysToFix) { - if (key.equals(".")) { - jsonObject.add("@", jsonObject.get(key)); - } else if (key.equals("..")) { - jsonObject.add("@@", jsonObject.get(key)); - } - jsonObject.remove(key); - } - return jsonObject; - } - @ProcessElement public void processElement(ProcessContext context) { PubsubMessage pubsubMessage = context.element().getOriginalPayload(); @@ -90,9 +54,7 @@ public void processElement(ProcessContext context) { try { if (pubsubMessage.getPayload().length > 0) { - messageObject = - patchJsonFields( - gson.fromJson(new String(pubsubMessage.getPayload()), JsonObject.class)); + messageObject = gson.fromJson(new String(pubsubMessage.getPayload()), JsonObject.class); } // If message attributes are present they will be serialized along with the message payload From fb68cfa76b5f67c13d9e8bbe51f6e9cbe0d2a4b6 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Sat, 7 May 2022 07:04:31 -0700 Subject: [PATCH 122/145] Resolve dependency conflicts. Also remove obsolete repositories from the project. PiperOrigin-RevId: 447184523 --- it/pom.xml | 4 ++-- pom.xml | 5 ----- v2/common/pom.xml | 4 ++++ v2/googlecloud-to-googlecloud/pom.xml | 28 +++++++++++++++++---------- v2/pom.xml | 5 ----- 5 files changed, 24 insertions(+), 22 deletions(-) diff --git a/it/pom.xml b/it/pom.xml index 4d186fba66..a77d7561c3 100644 --- a/it/pom.xml +++ b/it/pom.xml @@ -23,7 +23,7 @@ 1.8.2 - 2.34.0 + 2.38.0 24.2.0 2.2.2 1.8 @@ -211,4 +211,4 @@ - \ No newline at end of file + diff --git a/pom.xml b/pom.xml index 992a340cf0..33c51630e5 100644 --- a/pom.xml +++ b/pom.xml @@ -975,11 +975,6 @@ confluent https://packages.confluent.io/maven/ - - - pentaho - https://public.nexus.pentaho.org/repository/proxy-public-3rd-party-release - diff --git a/v2/common/pom.xml b/v2/common/pom.xml index 3d7e167ccd..31e19b535a 100644 --- a/v2/common/pom.xml +++ b/v2/common/pom.xml @@ -100,6 +100,10 @@ com.google.protobuf * + + com.google.apis + google-api-services-bigquery + diff --git a/v2/googlecloud-to-googlecloud/pom.xml b/v2/googlecloud-to-googlecloud/pom.xml index 18d90bb187..9bec734432 100644 --- a/v2/googlecloud-to-googlecloud/pom.xml +++ b/v2/googlecloud-to-googlecloud/pom.xml @@ -61,16 +61,6 @@ org.apache.beam beam-sdks-java-io-parquet - - org.apache.beam - beam-sdks-java-io-google-cloud-platform - - - io.grpc - grpc-netty-shaded - - - org.apache.beam beam-sdks-java-io-hadoop-common @@ -109,6 +99,12 @@ com.google.cloud google-cloud-kms ${kms.version} + + + com.google.cloud + google-cloud-core-grpc + + org.apache.hadoop @@ -123,6 +119,10 @@ org.slf4j slf4j-log4j12 + + com.google.protobuf + protobuf-java + @@ -138,6 +138,14 @@ org.slf4j slf4j-log4j12 + + com.google.protobuf + protobuf-java + + + com.google.code.gson + gson + diff --git a/v2/pom.xml b/v2/pom.xml index 197b1b5c84..f80cc1d480 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -665,11 +665,6 @@ confluent https://packages.confluent.io/maven/ - - - pentaho - https://public.nexus.pentaho.org/repository/proxy-public-3rd-party-release - From aef4401a76ea9eea17c6e558bc753fa37c2cf20a Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 9 May 2022 11:59:18 -0700 Subject: [PATCH 123/145] Some fixes for Spanner change streams to BigQuery Dataflow template PiperOrigin-RevId: 447528974 --- ...SpannerChangeStreamsToBigQueryOptions.java | 20 +++- .../FailsafeModJsonToTableRowTransformer.java | 111 +++++++++++++++--- .../SpannerChangeStreamsToBigQuery.java | 26 +++- 3 files changed, 133 insertions(+), 24 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java index d9ae1d9435..8d58517b89 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java @@ -27,6 +27,14 @@ */ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineOptions { + @Description( + "Project to read change streams from. The default for this parameter is the project where the" + + " Dataflow pipeline is running.") + @Default.String("") + String getSpannerProjectId(); + + void setSpannerProjectId(String projectId); + @Description("The Spanner instance ID that contains the Change Stream.") @Validation.Required String getSpannerInstanceId(); @@ -51,11 +59,19 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setSpannerMetadataDatabaseId(String value); + @Description( + "The Cloud Spanner change streams Connector metadata table name to use. If not provided, a" + + " Cloud Spanner change streams Connector metadata table will automatically be created" + + " during the pipeline flow.") + String getSpannerMetadataTableName(); + + void setSpannerMetadataTableName(String value); + @Description("The name of the Spanner Change Stream.") @Validation.Required - String getSpannerChangeStream(); + String getSpannerChangeStreamName(); - void setSpannerChangeStream(String value); + void setSpannerChangeStreamName(String value); @Description( "Priority for Spanner RPC invocations. Defaults to HIGH. Allowed priorites are LOW, MEDIUM," diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java index a8b358c621..738e7ecb34 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java @@ -15,12 +15,16 @@ */ package com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery; +import com.google.api.gax.grpc.GrpcCallContext; +import com.google.api.gax.rpc.ApiCallContext; import com.google.api.services.bigquery.model.TableRow; import com.google.auto.value.AutoValue; import com.google.cloud.spanner.Key.Builder; import com.google.cloud.spanner.KeySet; import com.google.cloud.spanner.Options; import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.SpannerOptions; +import com.google.cloud.spanner.SpannerOptions.CallContextConfigurator; import com.google.cloud.spanner.TimestampBound; import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.model.Mod; @@ -30,11 +34,17 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; import com.google.cloud.teleport.v2.values.FailsafeElement; +import io.grpc.CallOptions; +import io.grpc.Context; +import io.grpc.MethodDescriptor; +import java.io.IOException; +import java.io.ObjectInputStream; import java.io.Serializable; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; @@ -50,6 +60,8 @@ import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.node.ObjectNode; import org.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Class {@link FailsafeModJsonToTableRowTransformer} provides methods that convert a {@link Mod} @@ -57,6 +69,9 @@ */ public final class FailsafeModJsonToTableRowTransformer { + private static final Logger LOG = + LoggerFactory.getLogger(FailsafeModJsonToTableRowTransformer.class); + /** * Primary class for taking a {@link FailsafeElement} {@link Mod} JSON input and converting to a * {@link TableRow}. @@ -107,6 +122,7 @@ public static class FailsafeModJsonToTableRowFn private final Set ignoreFields; public TupleTag transformOut; public TupleTag> transformDeadLetterOut; + private transient CallContextConfigurator callContextConfigurator; public FailsafeModJsonToTableRowFn( SpannerConfig spannerConfig, @@ -124,12 +140,30 @@ public FailsafeModJsonToTableRowFn( } } + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); + setUpCallContextConfigurator(); + } + + private void setUpCallContextConfigurator() { + callContextConfigurator = + new CallContextConfigurator() { + public ApiCallContext configure( + ApiCallContext context, ReqT request, MethodDescriptor method) { + return GrpcCallContext.createDefault() + .withCallOptions(CallOptions.DEFAULT.withDeadlineAfter(120L, TimeUnit.SECONDS)); + } + }; + } + @Setup public void setUp() { spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); spannerTableByName = new SpannerUtils(spannerAccessor.getDatabaseClient(), spannerChangeStream) .getSpannerTableByName(); + setUpCallContextConfigurator(); } @Teardown @@ -230,26 +264,71 @@ private TableRow modJsonStringToTableRow(String modJsonString) throws Exception .map(spannerNonPkColumn -> spannerNonPkColumn.getName()) .collect(Collectors.toList()); - Options.ReadQueryUpdateTransactionOption options = - Options.priority(spannerConfig.getRpcPriority().get()); - // We assume the Spanner schema isn't changed while the pipeline is running, so the read is - // expected to succeed in normal cases. The schema change is currently not supported. - try (ResultSet resultSet = - spannerAccessor - .getDatabaseClient() - .singleUseReadOnlyTransaction( - TimestampBound.ofReadTimestamp(spannerCommitTimestamp)) - .read( - spannerTable.getTableName(), - KeySet.singleKey(keyBuilder.build()), - spannerNonPkColumnNames, - options)) { - SpannerToBigQueryUtils.spannerSnapshotRowToBigQueryTableRow( - resultSet, spannerNonPkColumns, tableRow); + int retryCount = 0; + while (true) { + try { + readSpannerRow( + spannerTable.getTableName(), + keyBuilder.build(), + spannerNonPkColumns, + spannerNonPkColumnNames, + spannerCommitTimestamp, + tableRow); + break; + } catch (Exception e) { + // Retry for maximum 3 times in case of transient error. + if (retryCount > 3) { + throw e; + } else { + LOG.error( + "Caught exception from Spanner snapshot read: {}, stack trace:{} current retry" + + " count: {}", + e, + e.getStackTrace(), + retryCount); + // Wait for 1 seconds before next retry. + TimeUnit.SECONDS.sleep(1); + retryCount++; + } + } } return tableRow; } + + // Do a Spanner read to retrieve full row. The schema change is currently not supported. so we + // assume the schema isn't changed while the pipeline is running, + private void readSpannerRow( + String spannerTableName, + com.google.cloud.spanner.Key key, + List spannerNonPkColumns, + List spannerNonPkColumnNames, + com.google.cloud.Timestamp spannerCommitTimestamp, + TableRow tableRow) { + Options.ReadQueryUpdateTransactionOption options = + Options.priority(spannerConfig.getRpcPriority().get()); + // Create a context that uses the custom call configuration. + Context context = + Context.current() + .withValue(SpannerOptions.CALL_CONTEXT_CONFIGURATOR_KEY, callContextConfigurator); + // Do the snapshot read in the custom context. + context.run( + () -> { + try (ResultSet resultSet = + spannerAccessor + .getDatabaseClient() + .singleUseReadOnlyTransaction( + TimestampBound.ofReadTimestamp(spannerCommitTimestamp)) + .read( + spannerTableName, + KeySet.singleKey(key), + spannerNonPkColumnNames, + options)) { + SpannerToBigQueryUtils.spannerSnapshotRowToBigQueryTableRow( + resultSet, spannerNonPkColumns, tableRow); + } + }); + } } } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java index 690a181ec9..af8a8e1c42 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java @@ -149,6 +149,7 @@ public static PipelineResult run(SpannerChangeStreamsToBigQueryOptions options) */ Pipeline pipeline = Pipeline.create(options); DeadLetterQueueManager dlqManager = buildDlqManager(options); + String spannerProjectId = getSpannerProjectId(options); String dlqDirectory = dlqManager.getRetryDlqDirectoryWithDateTime(); String tempDlqDirectory = dlqManager.getRetryDlqDirectory() + "tmp/"; @@ -166,7 +167,7 @@ public static PipelineResult run(SpannerChangeStreamsToBigQueryOptions options) SpannerConfig spannerConfig = SpannerConfig.create() .withHost(ValueProvider.StaticValueProvider.of(options.getSpannerHost())) - .withProjectId(options.getProject()) + .withProjectId(spannerProjectId) .withInstanceId(options.getSpannerInstanceId()) .withDatabaseId(options.getSpannerDatabaseId()) .withRpcPriority(options.getSpannerRpcPriority()); @@ -176,11 +177,16 @@ public static PipelineResult run(SpannerChangeStreamsToBigQueryOptions options) .withSpannerConfig(spannerConfig) .withMetadataInstance(options.getSpannerMetadataInstanceId()) .withMetadataDatabase(options.getSpannerMetadataDatabaseId()) - .withChangeStreamName(options.getSpannerChangeStream()) + .withChangeStreamName(options.getSpannerChangeStreamName()) .withInclusiveStartAt(startTimestamp) .withInclusiveEndAt(endTimestamp) .withRpcPriority(options.getSpannerRpcPriority()); + String spannerMetadataTableName = options.getSpannerMetadataTableName(); + if (spannerMetadataTableName != null) { + readChangeStream = readChangeStream.withMetadataTable(spannerMetadataTableName); + } + PCollection dataChangeRecord = pipeline .apply("Read from Spanner Change Streams", readChangeStream) @@ -217,7 +223,7 @@ public void process( failsafeModJsonToTableRowOptions = FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions.builder() .setSpannerConfig(spannerConfig) - .setSpannerChangeStream(options.getSpannerChangeStream()) + .setSpannerChangeStream(options.getSpannerChangeStreamName()) .setIgnoreFields(options.getIgnoreFields()) .setCoder(FAILSAFE_ELEMENT_CODER) .build(); @@ -232,8 +238,8 @@ public void process( bigQueryDynamicDestinationsOptions = BigQueryDynamicDestinations.BigQueryDynamicDestinationsOptions.builder() .setSpannerConfig(spannerConfig) - .setChangeStreamName(options.getSpannerChangeStream()) - .setBigQueryProject(getBigQueryProject(options)) + .setChangeStreamName(options.getSpannerChangeStreamName()) + .setBigQueryProject(getBigQueryProjectId(options)) .setBigQueryDataset(options.getBigQueryDataset()) .setBigQueryTableTemplate(options.getBigQueryChangelogTableNameTemplate()) .build(); @@ -274,6 +280,7 @@ public void process( DLQWriteTransform.WriteDLQ.newBuilder() .withDlqDirectory(dlqDirectory) .withTmpDirectory(tempDlqDirectory) + .setIncludePaneInfo(true) .build()); PCollection> nonRetryableDlqModJsonFailsafe = @@ -288,6 +295,7 @@ public void process( DLQWriteTransform.WriteDLQ.newBuilder() .withDlqDirectory(dlqManager.getSevereDlqDirectoryWithDateTime()) .withTmpDirectory(dlqManager.getSevereDlqDirectory() + "tmp/") + .setIncludePaneInfo(true) .build()); return pipeline.run(); @@ -306,7 +314,13 @@ private static DeadLetterQueueManager buildDlqManager( return DeadLetterQueueManager.create(dlqDirectory, DLQ_MAX_RETRIES); } - private static String getBigQueryProject(SpannerChangeStreamsToBigQueryOptions options) { + private static String getSpannerProjectId(SpannerChangeStreamsToBigQueryOptions options) { + return options.getSpannerProjectId().isEmpty() + ? options.getProject() + : options.getSpannerProjectId(); + } + + private static String getBigQueryProjectId(SpannerChangeStreamsToBigQueryOptions options) { return options.getBigQueryProjectId().isEmpty() ? options.getProject() : options.getBigQueryProjectId(); From 9bdc0cb50db5fc7cc3562883f32ee2f08b6b38fe Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 9 May 2022 17:00:35 -0700 Subject: [PATCH 124/145] Support for Cloud Spanner TABLE in PostgreSQL Interface PiperOrigin-RevId: 447597131 --- .../cloud/teleport/spanner/ddl/Column.java | 11 +- .../cloud/teleport/spanner/ddl/Ddl.java | 33 +++- .../spanner/ddl/DdlUtilityComponents.java | 13 ++ .../cloud/teleport/spanner/ddl/Dialect.java | 23 +++ .../cloud/teleport/spanner/ddl/Index.java | 28 +++- .../teleport/spanner/ddl/IndexColumn.java | 19 ++- .../cloud/teleport/spanner/ddl/Table.java | 107 ++++++++++-- .../cloud/teleport/spanner/ddl/DdlTest.java | 153 ++++++++++++++++++ 8 files changed, 357 insertions(+), 30 deletions(-) create mode 100644 src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java index 5ba72add66..942d214278 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java @@ -45,8 +45,11 @@ public abstract class Column implements Serializable { public abstract boolean isStored(); - public static Builder builder() { + public abstract Dialect dialect(); + + public static Builder builder(Dialect dialect) { return new AutoValue_Column.Builder() + .dialect(dialect) .columnOptions(ImmutableList.of()) .notNull(false) .isGenerated(false) @@ -54,6 +57,10 @@ public static Builder builder() { .isStored(false); } + public static Builder builder() { + return builder(Dialect.GOOGLE_STANDARD_SQL); + } + public void prettyPrint(Appendable appendable) throws IOException { appendable.append(String.format("%1$-40s", "`" + name() + "`")).append(typeString()); if (notNull()) { @@ -142,6 +149,8 @@ Builder tableBuilder(Table.Builder tableBuilder) { public abstract Builder notNull(boolean nullable); + abstract Builder dialect(Dialect dialect); + public Builder notNull() { return notNull(true); } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java index 3437f89947..8e5bbcfa3b 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java @@ -50,6 +50,7 @@ public class Ddl implements Serializable { // This is only populated by InformationSchemaScanner and not while reading from AVRO files. private TreeMultimap referencedTables; private final ImmutableList databaseOptions; + private final Dialect dialect; private Ddl( ImmutableSortedMap tables, @@ -57,13 +58,19 @@ private Ddl( ImmutableSortedMap changeStreams, TreeMultimap parents, TreeMultimap referencedTables, - ImmutableList databaseOptions) { + ImmutableList databaseOptions, + Dialect dialect) { this.tables = tables; this.views = views; this.changeStreams = changeStreams; this.parents = parents; this.referencedTables = referencedTables; this.databaseOptions = databaseOptions; + this.dialect = dialect; + } + + public Dialect dialect() { + return dialect; } public Collection allTables() { @@ -306,7 +313,11 @@ public String prettyPrint() { } public static Builder builder() { - return new Builder(); + return new Builder(Dialect.GOOGLE_STANDARD_SQL); + } + + public static Builder builder(Dialect dialect) { + return new Builder(dialect); } /** A builder for {@link Ddl}. */ @@ -318,11 +329,16 @@ public static class Builder { private TreeMultimap parents = TreeMultimap.create(); private TreeMultimap referencedTables = TreeMultimap.create(); private ImmutableList databaseOptions = ImmutableList.of(); + private Dialect dialect; + + public Builder(Dialect dialect) { + this.dialect = dialect; + } public Table.Builder createTable(String name) { Table table = tables.get(name.toLowerCase()); if (table == null) { - return Table.builder().name(name).ddlBuilder(this); + return Table.builder(dialect).name(name).ddlBuilder(this); } return table.toBuilder().ddlBuilder(this); } @@ -401,12 +417,13 @@ public Ddl build() { ImmutableSortedMap.copyOf(changeStreams), parents, referencedTables, - databaseOptions); + databaseOptions, + dialect); } } public Builder toBuilder() { - Builder builder = new Builder(); + Builder builder = new Builder(dialect); builder.tables.putAll(tables); builder.views.putAll(views); builder.changeStreams.putAll(changeStreams); @@ -427,6 +444,9 @@ public boolean equals(Object o) { Ddl ddl = (Ddl) o; + if (dialect != ddl.dialect) { + return false; + } if (tables != null ? !tables.equals(ddl.tables) : ddl.tables != null) { return false; } @@ -451,7 +471,8 @@ public boolean equals(Object o) { @Override public int hashCode() { - int result = tables != null ? tables.hashCode() : 0; + int result = dialect != null ? dialect.hashCode() : 0; + result = 31 * result + (tables != null ? tables.hashCode() : 0); result = 31 * result + (parents != null ? parents.hashCode() : 0); result = 31 * result + (referencedTables != null ? referencedTables.hashCode() : 0); result = 31 * result + (views != null ? views.hashCode() : 0); diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java index 849c11eb46..f889764d5a 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java @@ -33,4 +33,17 @@ private DdlUtilityComponents() {} .addEscape('\r', "\\r") .addEscape('\n', "\\n") .build(); + static final String POSTGRESQL_IDENTIFIER_QUOTE = "\""; + static final String GSQL_IDENTIFIER_QUOTE = "`"; + + static String identifierQuote(Dialect dialect) { + switch (dialect) { + case POSTGRESQL: + return POSTGRESQL_IDENTIFIER_QUOTE; + case GOOGLE_STANDARD_SQL: + return GSQL_IDENTIFIER_QUOTE; + default: + throw new IllegalArgumentException(String.format("Unrecognized dialect: %s", dialect)); + } + } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java new file mode 100644 index 0000000000..8a2ed8f3ff --- /dev/null +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java @@ -0,0 +1,23 @@ +/* + * Copyright 2021 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.ddl; + +/** Describe the dialect of the Cloud Spanner Database. */ +public enum Dialect { + GOOGLE_STANDARD_SQL, + POSTGRESQL; +} diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java index cbba31ba52..4dfda721ca 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java @@ -30,6 +30,8 @@ public abstract class Index implements Serializable { abstract String name(); + abstract Dialect dialect(); + abstract String table(); abstract ImmutableList indexColumns(); @@ -41,8 +43,12 @@ public abstract class Index implements Serializable { @Nullable abstract String interleaveIn(); + public static Builder builder(Dialect dialect) { + return new AutoValue_Index.Builder().dialect(dialect).nullFiltered(false).unique(false); + } + public static Builder builder() { - return new AutoValue_Index.Builder().nullFiltered(false).unique(false); + return builder(Dialect.GOOGLE_STANDARD_SQL); } public void prettyPrint(Appendable appendable) throws IOException { @@ -81,7 +87,7 @@ public void prettyPrint(Appendable appendable) throws IOException { public Builder toBuilder() { Builder builder = autoToBuilder(); for (IndexColumn column : indexColumns()) { - builder.columnsBuilder.set(column); + builder.columnsBuilder().set(column); } return builder; } @@ -105,17 +111,20 @@ public String toString() { @AutoValue.Builder public abstract static class Builder { - private IndexColumn.IndexColumnsBuilder columnsBuilder = - new IndexColumn.IndexColumnsBuilder<>(this); + private IndexColumn.IndexColumnsBuilder columnsBuilder; public abstract Builder name(String name); public abstract Builder table(String name); + abstract Builder dialect(Dialect dialect); + + public abstract Dialect dialect(); + abstract Builder indexColumns(ImmutableList columns); public IndexColumn.IndexColumnsBuilder columns() { - return columnsBuilder; + return columnsBuilder(); } public abstract Builder unique(boolean unique); @@ -135,7 +144,14 @@ public Builder nullFiltered() { abstract Index autoBuild(); public Index build() { - return this.indexColumns(columnsBuilder.build()).autoBuild(); + return this.indexColumns(columnsBuilder().build()).autoBuild(); + } + + private IndexColumn.IndexColumnsBuilder columnsBuilder() { + if (columnsBuilder == null) { + columnsBuilder = new IndexColumn.IndexColumnsBuilder<>(this, dialect()); + } + return columnsBuilder; } } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java index c122edc3a9..def1040ec4 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java @@ -30,8 +30,14 @@ public abstract class IndexColumn implements Serializable { public abstract Order order(); + public abstract Dialect dialect(); + + public static IndexColumn create(String name, Order order, Dialect dialect) { + return new AutoValue_IndexColumn(name, order, dialect); + } + public static IndexColumn create(String name, Order order) { - return new AutoValue_IndexColumn(name, order); + return new AutoValue_IndexColumn(name, order, Dialect.GOOGLE_STANDARD_SQL); } /** Ordering of column in the index. */ @@ -76,12 +82,15 @@ public static class IndexColumnsBuilder { private T callback; - public IndexColumnsBuilder(T callback) { + private Dialect dialect; + + public IndexColumnsBuilder(T callback, Dialect dialect) { this.callback = callback; + this.dialect = dialect; } public IndexColumnsBuilder asc(String name) { - IndexColumn indexColumn = IndexColumn.create(name, Order.ASC); + IndexColumn indexColumn = IndexColumn.create(name, Order.ASC, dialect); return set(indexColumn); } @@ -91,11 +100,11 @@ public IndexColumnsBuilder set(IndexColumn indexColumn) { } public IndexColumnsBuilder desc(String name) { - return set(IndexColumn.create(name, Order.DESC)); + return set(IndexColumn.create(name, Order.DESC, dialect)); } public IndexColumnsBuilder storing(String name) { - return set(IndexColumn.create(name, Order.STORING)); + return set(IndexColumn.create(name, Order.STORING, dialect)); } public ImmutableList build() { diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java index 414989ff60..d8208867bd 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java @@ -50,28 +50,55 @@ public abstract class Table implements Serializable { public abstract Builder autoToBuilder(); + public abstract Dialect dialect(); + public Builder toBuilder() { Builder builder = autoToBuilder(); + builder = builder.dialect(dialect()); for (Column column : columns()) { builder.addColumn(column); } for (IndexColumn pk : primaryKeys()) { - builder.primaryKeyBuilder.set(pk); + builder.primaryKeyBuilder().set(pk); } return builder; } - public static Builder builder() { + public static Builder builder(Dialect dialect) { return new AutoValue_Table.Builder() + .dialect(dialect) .indexes(ImmutableList.of()) .foreignKeys(ImmutableList.of()) .checkConstraints(ImmutableList.of()) .onDeleteCascade(false); } + public static Builder builder() { + return builder(Dialect.GOOGLE_STANDARD_SQL); + } + public void prettyPrint(Appendable appendable, boolean includeIndexes, boolean includeForeignKeys) throws IOException { - appendable.append("CREATE TABLE `").append(name()).append("` ("); + switch (dialect()) { + case GOOGLE_STANDARD_SQL: + prettyPrintGsql(appendable, includeIndexes, includeForeignKeys); + break; + case POSTGRESQL: + prettyPrintPg(appendable, includeIndexes, includeForeignKeys); + break; + default: + throw new IllegalArgumentException(String.format("Unrecognized Dialect: %s", dialect())); + } + } + + private void prettyPrintPg( + Appendable appendable, boolean includeIndexes, boolean includeForeignKeys) + throws IOException { + String identifierQuote = DdlUtilityComponents.identifierQuote(Dialect.POSTGRESQL); + appendable + .append("CREATE TABLE " + identifierQuote) + .append(name()) + .append(identifierQuote + " ("); for (Column column : columns()) { appendable.append("\n\t"); column.prettyPrint(appendable); @@ -82,7 +109,50 @@ public void prettyPrint(Appendable appendable, boolean includeIndexes, boolean i appendable.append(checkConstraint); appendable.append(","); } + if (primaryKeys() != null) { + appendable.append( + primaryKeys().stream() + .map(c -> identifierQuote + c.name() + identifierQuote) + .collect(Collectors.joining(", ", "\n\tPRIMARY KEY (", ")"))); + } + appendable.append("\n)"); + if (interleaveInParent() != null) { + appendable + .append(" \nINTERLEAVE IN PARENT " + identifierQuote) + .append(interleaveInParent()) + .append(identifierQuote); + if (onDeleteCascade()) { + appendable.append(" ON DELETE CASCADE"); + } + } + if (includeIndexes) { + appendable.append("\n"); + appendable.append(String.join("\n", indexes())); + } + if (includeForeignKeys) { + appendable.append("\n"); + appendable.append(String.join("\n", foreignKeys())); + } + } + private void prettyPrintGsql( + Appendable appendable, boolean includeIndexes, boolean includeForeignKeys) + throws IOException { + String identifierQuote = DdlUtilityComponents.identifierQuote(Dialect.GOOGLE_STANDARD_SQL); + appendable + .append("CREATE TABLE " + identifierQuote) + .append(name()) + .append(identifierQuote + " ("); + for (Column column : columns()) { + appendable.append("\n\t"); + column.prettyPrint(appendable); + appendable.append(","); + } + for (String checkConstraint : checkConstraints()) { + appendable.append("\n\t"); + appendable.append(checkConstraint); + appendable.append(","); + } if (primaryKeys() != null) { appendable.append( primaryKeys().stream() @@ -91,11 +161,14 @@ public void prettyPrint(Appendable appendable, boolean includeIndexes, boolean i } appendable.append(")"); if (interleaveInParent() != null) { - appendable.append(",\nINTERLEAVE IN PARENT `").append(interleaveInParent()).append("`"); - if (onDeleteCascade()) { - appendable.append(" ON DELETE CASCADE"); + appendable + .append(",\nINTERLEAVE IN PARENT " + identifierQuote) + .append(interleaveInParent()) + .append(identifierQuote); + if (onDeleteCascade()) { + appendable.append(" ON DELETE CASCADE"); + } } - } if (includeIndexes) { appendable.append("\n"); appendable.append(String.join("\n", indexes())); @@ -126,8 +199,7 @@ public String toString() { public abstract static class Builder { private Ddl.Builder ddlBuilder; - private IndexColumn.IndexColumnsBuilder primaryKeyBuilder = - new IndexColumn.IndexColumnsBuilder<>(this); + private IndexColumn.IndexColumnsBuilder primaryKeyBuilder; private LinkedHashMap columns = Maps.newLinkedHashMap(); Builder ddlBuilder(Ddl.Builder ddlBuilder) { @@ -153,10 +225,14 @@ Builder ddlBuilder(Ddl.Builder ddlBuilder) { public abstract Builder checkConstraints(ImmutableList checkConstraints); + abstract Builder dialect(Dialect dialect); + + public abstract Dialect dialect(); + abstract ImmutableList columns(); public IndexColumn.IndexColumnsBuilder primaryKey() { - return primaryKeyBuilder; + return primaryKeyBuilder(); } public Column.Builder column(String name) { @@ -164,7 +240,7 @@ public Column.Builder column(String name) { if (column != null) { return column.toBuilder().tableBuilder(this); } - return Column.builder().name(name).tableBuilder(this); + return Column.builder(dialect()).name(name).tableBuilder(this); } public Builder addColumn(Column column) { @@ -180,7 +256,7 @@ public Builder onDeleteCascade() { abstract Table autoBuild(); public Table build() { - return primaryKeys(primaryKeyBuilder.build()) + return primaryKeys(primaryKeyBuilder().build()) .columns(ImmutableList.copyOf(columns.values())) .autoBuild(); } @@ -189,6 +265,13 @@ public Ddl.Builder endTable() { ddlBuilder.addTable(build()); return ddlBuilder; } + + private IndexColumn.IndexColumnsBuilder primaryKeyBuilder() { + if (primaryKeyBuilder == null) { + primaryKeyBuilder = new IndexColumn.IndexColumnsBuilder<>(this, dialect()); + } + return primaryKeyBuilder; + } } public Column column(String name) { diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index 27f8953959..adace3b764 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.is; import static org.hamcrest.text.IsEqualCompressingWhiteSpace.equalToCompressingWhiteSpace; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import com.google.cloud.teleport.spanner.ExportProtos.Export; @@ -38,6 +39,14 @@ public void emptyDb() { assertThat(empty.prettyPrint(), equalTo("")); } + @Test + public void pgEmptyDb() { + Ddl empty = Ddl.builder(Dialect.POSTGRESQL).build(); + assertEquals(empty.dialect(), Dialect.POSTGRESQL); + assertThat(empty.allTables(), empty()); + assertThat(empty.prettyPrint(), equalTo("")); + } + @Test public void simple() { Ddl.Builder builder = Ddl.builder(); @@ -97,6 +106,68 @@ public void simple() { + " REFERENCES `AllowedNames` (`first_name`)")); } + @Test + public void pgSimple() { + Ddl.Builder builder = Ddl.builder(Dialect.POSTGRESQL); + builder + .createTable("Users") + .column("id") + .int64() + .notNull() + .endColumn() + .column("first_name") + .string() + .size(10) + .endColumn() + .column("last_name") + .type(Type.string()) + .max() + .endColumn() + .column("full_name") + .type(Type.string()) + .max() + .generatedAs("CONCAT(first_name, ' ', last_name)") + .stored() + .endColumn() + .primaryKey() + .asc("id") + .end() + .indexes( + ImmutableList.of("CREATE INDEX \"UsersByFirstName\" ON \"Users\" (\"first_name\")")) + .foreignKeys( + ImmutableList.of( + "ALTER TABLE \"Users\" ADD CONSTRAINT \"fk\" FOREIGN KEY (\"first_name\")" + + " REFERENCES \"AllowedNames\" (\"first_name\")")) + .checkConstraints( + ImmutableList.of("CONSTRAINT \"ck\" CHECK (\"first_name\" != \"last_name\")")) + .endTable(); + Export export = + Export.newBuilder() + .addDatabaseOptions( + Export.DatabaseOption.newBuilder() + .setOptionName("version_retention_period") + .setOptionValue("4d") + .build()) + .build(); + builder.mergeDatabaseOptions(export.getDatabaseOptionsList()); + Ddl ddl = builder.build(); + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "ALTER DATABASE `%db_name%` SET OPTIONS ( version_retention_period = 4d )" + + " CREATE TABLE \"Users\" (" + + " `id` INT64 NOT NULL," + + " `first_name` STRING(10)," + + " `last_name` STRING(MAX)," + + " `full_name` STRING(MAX) AS (CONCAT(first_name, ' ', last_name)) STORED," + + " CONSTRAINT \"ck\" CHECK (\"first_name\" != \"last_name\")," + + " PRIMARY KEY (\"id\")" + + " ) " + + " CREATE INDEX \"UsersByFirstName\" ON \"Users\" (\"first_name\")" + + " ALTER TABLE \"Users\" ADD CONSTRAINT \"fk\" FOREIGN KEY (\"first_name\")" + + " REFERENCES \"AllowedNames\" (\"first_name\")")); + } + @Test public void interleaves() { Ddl ddl = @@ -154,6 +225,65 @@ public void interleaves() { + " INTERLEAVE IN PARENT `Users` ON DELETE CASCADE")); } + @Test + public void pgInterleaves() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("id") + .int64() + .notNull() + .endColumn() + .column("first_name") + .string() + .size(10) + .endColumn() + .column("last_name") + .type(Type.string()) + .max() + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .createTable("Account") + .column("id") + .int64() + .notNull() + .endColumn() + .column("balanceId") + .int64() + .notNull() + .endColumn() + .column("balance") + .float64() + .notNull() + .endColumn() + .primaryKey() + .asc("id") + .end() + .interleaveInParent("Users") + .onDeleteCascade() + .endTable() + .build(); + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE TABLE \"Users\" (" + + " `id` INT64 NOT NULL," + + " `first_name` STRING(10)," + + " `last_name` STRING(MAX)," + + " PRIMARY KEY (\"id\")" + + " ) " + + " CREATE TABLE \"Account\" (" + + " `id` INT64 NOT NULL," + + " `balanceId` INT64 NOT NULL," + + " `balance` FLOAT64 NOT NULL," + + " PRIMARY KEY (\"id\")" + + " ) " + + " INTERLEAVE IN PARENT \"Users\" ON DELETE CASCADE")); + } + @Test public void testDatabaseOptions() { Ddl.Builder builder = Ddl.builder(); @@ -177,6 +307,29 @@ public void testDatabaseOptions() { is("ALTER DATABASE `database_id` SET OPTIONS ( version_retention_period = 4d )")); } + @Test + public void pgTestDatabaseOptions() { + Ddl.Builder builder = Ddl.builder(Dialect.POSTGRESQL); + List dbOptionList = new ArrayList<>(); + dbOptionList.add( + Export.DatabaseOption.newBuilder() + .setOptionName("version_retention_period") + .setOptionValue("4d") + .build()); + dbOptionList.add( + Export.DatabaseOption.newBuilder() + .setOptionName("optimizer_version") + .setOptionValue("2") + .build()); + builder.mergeDatabaseOptions(dbOptionList); + Ddl ddl = builder.build(); + List optionStatements = ddl.setOptionsStatements("database_id"); + assertThat(optionStatements.size(), is(1)); + assertThat( + optionStatements.get(0), + is("ALTER DATABASE `database_id` SET OPTIONS ( version_retention_period = 4d )")); + } + @Test public void changeStreams() { Ddl ddl = From 5355784a7a3eb749a24b598cc387af8184683249 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Mon, 9 May 2022 17:25:24 -0700 Subject: [PATCH 125/145] Submit integration test for change stream to GCS dataflow template PiperOrigin-RevId: 447602211 --- v2/googlecloud-to-googlecloud/pom.xml | 22 +++ ...FileFormatFactorySpannerChangeStreams.java | 4 +- .../WriteDataChangeRecordsToGcsAvro.java | 186 ++++++++++++++++-- .../WriteDataChangeRecordsToGcsText.java | 21 +- .../schema/avro/datachangerecord.avsc | 88 +++++++++ .../SpannerChangeStreamsToGcsTest.java | 35 ++-- .../WriteDataChangeRecordsToGcsAvroTest.java | 17 +- .../WriteDataChangeRecordsToGcsTextTest.java | 8 +- 8 files changed, 326 insertions(+), 55 deletions(-) create mode 100644 v2/googlecloud-to-googlecloud/src/main/resources/schema/avro/datachangerecord.avsc diff --git a/v2/googlecloud-to-googlecloud/pom.xml b/v2/googlecloud-to-googlecloud/pom.xml index 9bec734432..dd6fe6ff0c 100644 --- a/v2/googlecloud-to-googlecloud/pom.xml +++ b/v2/googlecloud-to-googlecloud/pom.xml @@ -36,6 +36,7 @@ 3.0.0 10.14.2.0 2.8.5 + 1.8.2 @@ -53,6 +54,27 @@ + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + ${project.basedir}/src/main/resources/schema/avro + + + + generate-sources + + schema + + + ${project.basedir}/src/main/resources/schema/avro + + + + diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java index c669b8d57a..634de502ef 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/FileFormatFactorySpannerChangeStreams.java @@ -62,7 +62,7 @@ public POutput expand(PCollection records) { records.apply( "Write Avro File(s)", WriteDataChangeRecordsToGcsAvro.newBuilder() - .withOutputDirectory(options().getOutputDirectory()) + .withGcsOutputDirectory(options().getGcsOutputDirectory()) .withOutputFilenamePrefix(options().getOutputFilenamePrefix()) .setNumShards(options().getNumShards()) .withTempLocation(options().getTempLocation()) @@ -73,7 +73,7 @@ public POutput expand(PCollection records) { records.apply( "Write Text File(s)", WriteDataChangeRecordsToGcsText.newBuilder() - .withOutputDirectory(options().getOutputDirectory()) + .withGcsOutputDirectory(options().getGcsOutputDirectory()) .withOutputFilenamePrefix(options().getOutputFilenamePrefix()) .setNumShards(options().getNumShards()) .withTempLocation(options().getTempLocation()) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java index 8ea07c1d06..ba0a127536 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvro.java @@ -19,15 +19,24 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; +import com.google.cloud.Timestamp; import com.google.cloud.teleport.v2.io.WindowedFilenamePolicy; import com.google.cloud.teleport.v2.utils.WriteToGCSUtility; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ModType; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.TypeCode; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ValueCaptureType; 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.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; @@ -50,7 +59,7 @@ public static WriteToGcsBuilder newBuilder() { return new AutoValue_WriteDataChangeRecordsToGcsAvro.Builder(); } - public abstract String outputDirectory(); + public abstract String gcsOutputDirectory(); public abstract String outputFilenamePrefix(); @@ -68,19 +77,153 @@ public PDone expand(PCollection dataChangeRecords) { * The {@link withNumShards} option specifies the number of shards passed by the user. * The {@link withTempDirectory} option sets the base directory used to generate temporary files. */ + .apply("Transform to Avro", MapElements.via(new DataChangeRecordToAvroFn())) .apply( - "Writing as Avro", - AvroIO.write(DataChangeRecord.class) - .to( - new WindowedFilenamePolicy( - outputDirectory(), - outputFilenamePrefix(), - WriteToGCSUtility.SHARD_TEMPLATE, - WriteToGCSUtility.FILE_SUFFIX_MAP.get(WriteToGCSUtility.FileFormat.AVRO))) - .withTempDirectory( - FileBasedSink.convertToFileResourceIfPossible(tempLocation()).getCurrentDirectory()) - .withWindowedWrites() - .withNumShards(numShards())); + "Writing as Avro", + AvroIO.write(com.google.cloud.teleport.v2.DataChangeRecord.class) + .to( + new WindowedFilenamePolicy( + gcsOutputDirectory(), + outputFilenamePrefix(), + WriteToGCSUtility.SHARD_TEMPLATE, + WriteToGCSUtility.FILE_SUFFIX_MAP.get(WriteToGCSUtility.FileFormat.AVRO))) + .withTempDirectory( + FileBasedSink.convertToFileResourceIfPossible(tempLocation()) + .getCurrentDirectory()) + .withWindowedWrites() + .withNumShards(numShards())); + } + + private static long timestampToMicros(Timestamp ts) { + return TimeUnit.SECONDS.toMicros(ts.getSeconds()) + + TimeUnit.NANOSECONDS.toMicros(ts.getNanos()); + } + + public static com.google.cloud.teleport.v2.DataChangeRecord dataChangeRecordToAvro( + DataChangeRecord record) { + String partitionToken = record.getPartitionToken(); + long commitTimestampMicros = timestampToMicros(record.getCommitTimestamp()); + String serverTransactionId = record.getServerTransactionId(); + boolean isLastRecordInTransaction = record.isLastRecordInTransactionInPartition(); + String recordSequence = record.getRecordSequence(); + String tableName = record.getTableName(); + List columnTypes = + record.getRowType().stream() + .map( + columnType -> + new com.google.cloud.teleport.v2.ColumnType( + columnType.getName(), + mapTypeCodeToAvro(columnType.getType()), + columnType.isPrimaryKey(), + columnType.getOrdinalPosition())) + .collect(Collectors.toList()); + + List mods = + record.getMods().stream() + .map( + mod -> + new com.google.cloud.teleport.v2.Mod( + mod.getKeysJson(), + mod.getOldValuesJson() != null ? mod.getOldValuesJson() : "", + mod.getNewValuesJson() != null ? mod.getNewValuesJson() : "")) + .collect(Collectors.toList()); + + com.google.cloud.teleport.v2.ModType modType = mapModTypeToModTypeAvro(record.getModType()); + com.google.cloud.teleport.v2.ValueCaptureType captureType = + mapValueCaptureTypeToAvro(record.getValueCaptureType()); + long numberOfRecordsInTransaction = record.getNumberOfRecordsInTransaction(); + long numberOfPartitionsInTransaction = record.getNumberOfPartitionsInTransaction(); + + com.google.cloud.teleport.v2.ChangeStreamRecordMetadata metadata = + record.getMetadata() == null + ? null + : new com.google.cloud.teleport.v2.ChangeStreamRecordMetadata( + record.getMetadata().getPartitionToken(), + timestampToMicros(record.getMetadata().getRecordTimestamp()), + timestampToMicros(record.getMetadata().getPartitionStartTimestamp()), + timestampToMicros(record.getMetadata().getPartitionEndTimestamp()), + timestampToMicros(record.getMetadata().getPartitionCreatedAt()), + record.getMetadata().getPartitionScheduledAt() == null + ? 0 + : timestampToMicros(record.getMetadata().getPartitionScheduledAt()), + record.getMetadata().getPartitionRunningAt() == null + ? 0 + : timestampToMicros(record.getMetadata().getPartitionRunningAt()), + timestampToMicros(record.getMetadata().getQueryStartedAt()), + timestampToMicros(record.getMetadata().getRecordStreamStartedAt()), + timestampToMicros(record.getMetadata().getRecordStreamEndedAt()), + timestampToMicros(record.getMetadata().getRecordReadAt()), + record.getMetadata().getTotalStreamTimeMillis(), + record.getMetadata().getNumberOfRecordsRead()); + + // Add ChangeStreamMetadata + return new com.google.cloud.teleport.v2.DataChangeRecord( + partitionToken, + commitTimestampMicros, + serverTransactionId, + isLastRecordInTransaction, + recordSequence, + tableName, + columnTypes, + mods, + modType, + captureType, + numberOfRecordsInTransaction, + numberOfPartitionsInTransaction, + metadata); + } + + private static com.google.cloud.teleport.v2.ModType mapModTypeToModTypeAvro(ModType modType) { + switch (modType) { + case INSERT: + return com.google.cloud.teleport.v2.ModType.INSERT; + case UPDATE: + return com.google.cloud.teleport.v2.ModType.UPDATE; + default: + return com.google.cloud.teleport.v2.ModType.DELETE; + } + } + + private static com.google.cloud.teleport.v2.ValueCaptureType mapValueCaptureTypeToAvro( + ValueCaptureType modType) { + return com.google.cloud.teleport.v2.ValueCaptureType.OLD_AND_NEW_VALUES; + } + + private static com.google.cloud.teleport.v2.TypeCode mapTypeCodeToAvro(TypeCode typeCode) { + switch (typeCode.getCode()) { + case "{\"code\":\"BOOL\"}": + return com.google.cloud.teleport.v2.TypeCode.BOOL; + case "{\"code\":\"INT64\"}": + return com.google.cloud.teleport.v2.TypeCode.INT64; + case "{\"code\":\"FLOAT64\"}": + return com.google.cloud.teleport.v2.TypeCode.FLOAT64; + case "{\"code\":\"TIMESTAMP\"}": + return com.google.cloud.teleport.v2.TypeCode.TIMESTAMP; + case "{\"code\":\"DATE\"}": + return com.google.cloud.teleport.v2.TypeCode.DATE; + case "{\"code\":\"STRING\"}": + return com.google.cloud.teleport.v2.TypeCode.STRING; + case "{\"code\":\"BYTES\"}": + return com.google.cloud.teleport.v2.TypeCode.BYTES; + case "{\"code\":\"ARRAY\"}": + return com.google.cloud.teleport.v2.TypeCode.ARRAY; + case "{\"code\":\"STRUCT\"}": + return com.google.cloud.teleport.v2.TypeCode.STRUCT; + case "{\"code\":\"NUMERIC\"}": + return com.google.cloud.teleport.v2.TypeCode.NUMERIC; + case "{\"code\":\"JSON\"}": + return com.google.cloud.teleport.v2.TypeCode.JSON; + default: + return com.google.cloud.teleport.v2.TypeCode.TYPE_CODE_UNSPECIFIED; + } + } + + static class DataChangeRecordToAvroFn + extends SimpleFunction { + @Override + public com.google.cloud.teleport.v2.DataChangeRecord apply(DataChangeRecord record) { + return dataChangeRecordToAvro(record); + } } /** @@ -89,9 +232,9 @@ public PDone expand(PCollection dataChangeRecords) { */ public interface WriteToGcsAvroOptions extends PipelineOptions { @Description("The directory to output files to. Must end with a slash.") - String getOutputDirectory(); + String getGcsOutputDirectory(); - void setOutputDirectory(String outputDirectory); + void setGcsOutputDirectory(String gcsOutputDirectory); @Description( "The filename prefix of the files to write to. Default file prefix is set to \"output\".") @@ -112,9 +255,9 @@ public interface WriteToGcsAvroOptions extends PipelineOptions { /** Builder for {@link WriteDataChangeRecordsToGcsAvro}. */ @AutoValue.Builder public abstract static class WriteToGcsBuilder { - abstract WriteToGcsBuilder setOutputDirectory(String outputDirectory); + abstract WriteToGcsBuilder setGcsOutputDirectory(String gcsOutputDirectory); - abstract String outputDirectory(); + abstract String gcsOutputDirectory(); abstract WriteToGcsBuilder setTempLocation(String tempLocation); @@ -126,10 +269,11 @@ public abstract static class WriteToGcsBuilder { abstract WriteDataChangeRecordsToGcsAvro autoBuild(); - public WriteToGcsBuilder withOutputDirectory(String outputDirectory) { + public WriteToGcsBuilder withGcsOutputDirectory(String gcsOutputDirectory) { checkArgument( - outputDirectory != null, "withOutputDirectory(outputDirectory) called with null input."); - return setOutputDirectory(outputDirectory); + gcsOutputDirectory != null, + "withGcsOutputDirectory(gcsOutputDirectory) called with null input."); + return setGcsOutputDirectory(gcsOutputDirectory); } public WriteToGcsBuilder withTempLocation(String tempLocation) { @@ -146,7 +290,7 @@ public WriteToGcsBuilder withOutputFilenamePrefix(String outputFilenamePrefix) { } public WriteDataChangeRecordsToGcsAvro build() { - checkNotNull(outputDirectory(), "Provide output directory to write to. "); + checkNotNull(gcsOutputDirectory(), "Provide output directory to write to. "); checkNotNull(tempLocation(), "Temporary directory needs to be provided. "); return autoBuild(); } diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java index b9320933fa..7add081bca 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsText.java @@ -53,7 +53,7 @@ public static WriteToGcsBuilder newBuilder() { return new AutoValue_WriteDataChangeRecordsToGcsText.Builder(); } - public abstract String outputDirectory(); + public abstract String gcsOutputDirectory(); public abstract String outputFilenamePrefix(); @@ -81,7 +81,7 @@ public PDone expand(PCollection dataChangeRecords) { TextIO.write() .to( new WindowedFilenamePolicy( - outputDirectory(), + gcsOutputDirectory(), outputFilenamePrefix(), WriteToGCSUtility.SHARD_TEMPLATE, WriteToGCSUtility.FILE_SUFFIX_MAP.get(WriteToGCSUtility.FileFormat.TEXT))) @@ -107,9 +107,9 @@ public String apply(DataChangeRecord record) { */ public interface WriteToGcsTextOptions extends PipelineOptions { @Description("The directory to output files to. Must end with a slash.") - String getOutputDirectory(); + String getGcsOutputDirectory(); - void setOutputDirectory(String outputDirectory); + void setGcsOutputDirectory(String gcsOutputDirectory); @Description( "The filename prefix of the files to write to. Default file prefix is set to \"output\". ") @@ -130,9 +130,9 @@ public interface WriteToGcsTextOptions extends PipelineOptions { /** Builder for {@link WriteDataChangeRecordsToGcsText}. */ @AutoValue.Builder public abstract static class WriteToGcsBuilder { - abstract WriteToGcsBuilder setOutputDirectory(String outputDirectory); + abstract WriteToGcsBuilder setGcsOutputDirectory(String gcsOutputDirectory); - abstract String outputDirectory(); + abstract String gcsOutputDirectory(); abstract WriteToGcsBuilder setTempLocation(String tempLocation); @@ -144,10 +144,11 @@ public abstract static class WriteToGcsBuilder { abstract WriteDataChangeRecordsToGcsText autoBuild(); - public WriteToGcsBuilder withOutputDirectory(String outputDirectory) { + public WriteToGcsBuilder withGcsOutputDirectory(String gcsOutputDirectory) { checkArgument( - outputDirectory != null, "withOutputDirectory(outputDirectory) called with null input."); - return setOutputDirectory(outputDirectory); + gcsOutputDirectory != null, + "withGcsOutputDirectory(gcsOutputDirectory) called with null input."); + return setGcsOutputDirectory(gcsOutputDirectory); } public WriteToGcsBuilder withTempLocation(String tempLocation) { @@ -164,7 +165,7 @@ public WriteToGcsBuilder withOutputFilenamePrefix(String outputFilenamePrefix) { } public WriteDataChangeRecordsToGcsText build() { - checkNotNull(outputDirectory(), "Provide output directory to write to."); + checkNotNull(gcsOutputDirectory(), "Provide output directory to write to."); checkNotNull(tempLocation(), "Temporary directory needs to be provided."); return autoBuild(); } diff --git a/v2/googlecloud-to-googlecloud/src/main/resources/schema/avro/datachangerecord.avsc b/v2/googlecloud-to-googlecloud/src/main/resources/schema/avro/datachangerecord.avsc new file mode 100644 index 0000000000..1b2870b4ac --- /dev/null +++ b/v2/googlecloud-to-googlecloud/src/main/resources/schema/avro/datachangerecord.avsc @@ -0,0 +1,88 @@ +{ + "type": "record", + "name": "DataChangeRecord", + "namespace": "com.google.cloud.teleport.v2", + "fields" : [ + { "name" : "partitionToken", "type" : "string"}, + { "name" : "commitTimestamp", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "serverTransactionId", "type" : "string"}, + { "name" : "isLastRecordInTransactionInPartition", "type" : "boolean"}, + { "name" : "recordSequence", "type" : "string"}, + { "name" : "tableName", "type" : "string"}, + { "name" : "rowType", + "type" : { + "type" : "array", + "items": { + "name": "ColumnType", + "type": "record", + "fields": [ + { "name" : "name", "type" : "string"}, + { "name" : "Type", + "type" : { + "name": "TypeCode", + "type": "enum", + "symbols": ["TYPE_CODE_UNSPECIFIED", "BOOL", "INT64", "FLOAT64", + "TIMESTAMP", "DATE", "STRING", "BYTES", "ARRAY", "STRUCT", + "NUMERIC", "JSON"] + } + }, + { "name" : "isPrimaryKey", "type" : "boolean"}, + { "name" : "ordinalPosition", "type" : "long"} + ] + }, + "java-class":"java.util.List" + } + }, + { "name" : "mods", + "type" : { + "type" : "array", + "items": { + "name": "Mod", + "type": "record", + "fields": [ + { "name" : "keysJson", "type" : "string"}, + { "name" : "oldValuesJson", "type" : ["null", "string"], "default" : null}, + { "name" : "newValuesJson", "type": ["null", "string"], "default" : null} + ] + }, + "java-class":"java.util.List" + } + }, + { "name" : "modType", + "type" : { + "name": "ModType", + "type": "enum", + "symbols": ["INSERT", "UPDATE", "DELETE"] + } + }, + + { "name" : "valueCaptureType", + "type" : { + "name": "ValueCaptureType", + "type": "enum", + "symbols": ["OLD_AND_NEW_VALUES"] + } + }, + { "name" : "numberOfRecordsInTransaction", "type" : "long"}, + { "name" : "numberOfPartitionsInTransaction", "type" : "long"}, + { "name" : "metadata", + "type" : ["null", {"type" : "record", + "name" : "ChangeStreamRecordMetadata", + "fields" : [ + { "name" : "partitionToken", "type" : "string"}, + { "name" : "recordTimestamp", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "partitionStartTimestamp", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "partitionEndTimestamp", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "partitionCreatedAt", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "partitionScheduledAt", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "partitionRunningAt", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "queryStartedAt", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "recordStreamStartedAt", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "recordStreamEndedAt", "type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "recordReadAt","type" : "long", "logicalType" : "timestamp-micros"}, + { "name" : "totalStreamTimeMillis", "type":"long"}, + { "name" : "numberOfRecordsRead","type":"long"} + ] + }],"default":null} + ] + } \ No newline at end of file diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java index 8c7815415c..4503718f2d 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java @@ -17,6 +17,7 @@ import static com.google.cloud.teleport.v2.templates.SpannerChangeStreamsToGcs.run; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import com.google.cloud.Timestamp; import com.google.cloud.spanner.Mutation; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; -import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -88,17 +88,29 @@ public void setup() throws Exception { @SuppressWarnings("DefaultAnnotationParam") private static class VerifyDataChangeRecordAvro - implements SerializableFunction, Void> { + implements SerializableFunction< + Iterable, Void> { @Override - public Void apply(Iterable actualIter) { + public Void apply(Iterable actualIter) { // Make sure actual is the right length, and is a // subset of expected. - List actual = new ArrayList<>(); - for (DataChangeRecord s : actualIter) { + List actual = new ArrayList<>(); + for (com.google.cloud.teleport.v2.DataChangeRecord s : actualIter) { actual.add(s); assertEquals(TEST_TABLE, s.getTableName()); + assertTrue(s.getCommitTimestamp() > 0); + assertTrue(s.getPartitionToken() != null && s.getPartitionToken().length() > 0); + assertTrue(s.getServerTransactionId() != null && s.getServerTransactionId().length() > 0); + assertTrue(s.getRecordSequence() != null && s.getRecordSequence().length() > 0); + assertTrue(!s.getRowType().isEmpty()); + assertTrue( + s.getRowType().get(0).getType() + != com.google.cloud.teleport.v2.TypeCode.TYPE_CODE_UNSPECIFIED); + assertTrue(!s.getMods().isEmpty()); + assertTrue(s.getNumberOfRecordsInTransaction() > 0); + assertTrue(s.getNumberOfPartitionsInTransaction() > 0); + assertTrue(s.getMetadata() != null); } - assertEquals(actual.size(), 1); return null; } } @@ -123,7 +135,7 @@ public void testFileFormatFactoryInvalid() { SpannerChangeStreamsToGcsOptions options = PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); options.setOutputFileFormat(FileFormat.PARQUET); - options.setOutputDirectory(fakeDir); + options.setGcsOutputDirectory(fakeDir); options.setOutputFilenamePrefix(FILENAME_PREFIX); options.setNumShards(NUM_SHARDS); options.setTempLocation(fakeTempLocation); @@ -165,7 +177,7 @@ public void testInvalidWindowDuration() { SpannerChangeStreamsToGcsOptions options = PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); options.setOutputFileFormat(FileFormat.AVRO); - options.setOutputDirectory(fakeDir); + options.setGcsOutputDirectory(fakeDir); options.setOutputFilenamePrefix(FILENAME_PREFIX); options.setNumShards(NUM_SHARDS); options.setTempLocation(fakeTempLocation); @@ -258,7 +270,7 @@ public void testWriteToGCSAvro() throws Exception { options.setExperiments(experiments); options.setOutputFileFormat(FileFormat.AVRO); - options.setOutputDirectory(fakeDir); + options.setGcsOutputDirectory(fakeDir); options.setOutputFilenamePrefix(AVRO_FILENAME_PREFIX); options.setNumShards(NUM_SHARDS); options.setTempLocation(fakeTempLocation); @@ -268,10 +280,11 @@ public void testWriteToGCSAvro() throws Exception { result.waitUntilFinish(); // Read from the output Avro file to assert that 1 data change record has been generated. - PCollection dataChangeRecords = + PCollection dataChangeRecords = pipeline.apply( "readRecords", - AvroIO.read(DataChangeRecord.class).from(fakeDir + "/avro-output-*.avro")); + AvroIO.read(com.google.cloud.teleport.v2.DataChangeRecord.class) + .from(fakeDir + "/avro-output-*.avro")); PAssert.that(dataChangeRecords).satisfies(new VerifyDataChangeRecordAvro()); pipeline.run(); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java index 326c7fe54a..2b7a759abe 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsAvroTest.java @@ -73,7 +73,7 @@ public void testBasicWrite() { .apply( "WriteTextFile(s)", WriteDataChangeRecordsToGcsAvro.newBuilder() - .withOutputDirectory(fakeDir) + .withGcsOutputDirectory(fakeDir) .withOutputFilenamePrefix(AVRO_FILENAME_PREFIX) .setNumShards(NUM_SHARDS) .withTempLocation(fakeTempLocation) @@ -81,12 +81,14 @@ public void testBasicWrite() { p.run(); // Then, read the records back from the output directory using AvrioIO.read. - PCollection dataChangeRecords = + PCollection dataChangeRecords = pipeline.apply( "readRecords", - AvroIO.read(DataChangeRecord.class) + AvroIO.read(com.google.cloud.teleport.v2.DataChangeRecord.class) .from(fakeDir + "/avro-output-GlobalWindow-pane-0-last-00-of-01.avro")); - PAssert.that(dataChangeRecords).containsInAnyOrder(dataChangeRecord); + PAssert.that(dataChangeRecords) + .containsInAnyOrder( + WriteDataChangeRecordsToGcsAvro.dataChangeRecordToAvro(dataChangeRecord)); pipeline.run(); } @@ -97,14 +99,15 @@ public void testBasicWrite() { @Test public void testWriteWithoutOutputDirectory() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("withOutputDirectory(outputDirectory) called with null input."); + expectedException.expectMessage( + "withGcsOutputDirectory(gcsOutputDirectory) called with null input."); final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); pipeline .apply("CreateInput", Create.of(dataChangeRecord)) .apply( "WriteTextFile(s)", WriteDataChangeRecordsToGcsAvro.newBuilder() - .withOutputDirectory(null) + .withGcsOutputDirectory(null) .withOutputFilenamePrefix(AVRO_FILENAME_PREFIX) .setNumShards(NUM_SHARDS) .withTempLocation(fakeTempLocation) @@ -125,7 +128,7 @@ public void testWriteWithoutTempLocation() { .apply( "WriteTextFile(s)", WriteDataChangeRecordsToGcsAvro.newBuilder() - .withOutputDirectory(fakeDir) + .withGcsOutputDirectory(fakeDir) .withOutputFilenamePrefix(AVRO_FILENAME_PREFIX) .setNumShards(NUM_SHARDS) .withTempLocation(null) diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java index b5614a7cfa..e0f23bfe5a 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/transforms/WriteDataChangeRecordsToGcsTextTest.java @@ -75,7 +75,7 @@ public void testBasicWrite() { .apply( "WriteTextFile(s)", WriteDataChangeRecordsToGcsText.newBuilder() - .withOutputDirectory(fakeDir) + .withGcsOutputDirectory(fakeDir) .withOutputFilenamePrefix(TEXT_FILENAME_PREFIX) .setNumShards(NUM_SHARDS) .withTempLocation(fakeTempLocation) @@ -99,14 +99,14 @@ public void testBasicWrite() { @Test public void testWriteWithoutOutputDirectory() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("withOutputDirectory(outputDirectory) called with null input."); + expectedException.expectMessage("withGcsOutputDirectory(gcsOutputDirectory) called with null input."); final DataChangeRecord dataChangeRecord = createTestDataChangeRecord(); pipeline .apply("CreateInput", Create.of(dataChangeRecord)) .apply( "WriteTextFile(s)", WriteDataChangeRecordsToGcsText.newBuilder() - .withOutputDirectory(null) + .withGcsOutputDirectory(null) .withOutputFilenamePrefix(TEXT_FILENAME_PREFIX) .setNumShards(NUM_SHARDS) .withTempLocation(fakeTempLocation) @@ -127,7 +127,7 @@ public void testWriteWithoutTempLocation() { .apply( "WriteTextFile(s)", WriteDataChangeRecordsToGcsText.newBuilder() - .withOutputDirectory(fakeDir) + .withGcsOutputDirectory(fakeDir) .withOutputFilenamePrefix(TEXT_FILENAME_PREFIX) .setNumShards(NUM_SHARDS) .withTempLocation(null) From 3334e5193fa75276ab07713fd1b54dcbd270dc3f Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 00:09:17 -0700 Subject: [PATCH 126/145] Support PostgreSQL types in Cloud Spanner PosgreSQL interface. PiperOrigin-RevId: 447657902 --- .../teleport/spanner/common/NumericUtils.java | 51 ++++ .../cloud/teleport/spanner/common/Type.java | 224 ++++++++++-------- .../cloud/teleport/spanner/ddl/Column.java | 214 +++++++++++++---- src/main/proto/type.proto | 136 ----------- .../spanner/common/NumericUtilsTest.java | 42 ++++ .../cloud/teleport/spanner/ddl/DdlTest.java | 43 ++-- 6 files changed, 407 insertions(+), 303 deletions(-) delete mode 100644 src/main/proto/type.proto create mode 100644 src/test/java/com/google/cloud/teleport/spanner/common/NumericUtilsTest.java diff --git a/src/main/java/com/google/cloud/teleport/spanner/common/NumericUtils.java b/src/main/java/com/google/cloud/teleport/spanner/common/NumericUtils.java index 64b8319b89..25c45a1f93 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/common/NumericUtils.java +++ b/src/main/java/com/google/cloud/teleport/spanner/common/NumericUtils.java @@ -15,6 +15,11 @@ */ package com.google.cloud.teleport.spanner.common; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.math.MathContext; @@ -25,6 +30,9 @@ public final class NumericUtils { public static final int PRECISION = 38; public static final int SCALE = 9; + public static final int PG_MAX_PRECISION = 147455; + public static final int PG_MAX_SCALE = 16383; + // Convert an Avro-encoded NUMERIC byte array to an readable NUMERIC string value. public static String bytesToString(byte[] byteArray) { BigInteger unscaledNumeric = new BigInteger(byteArray); @@ -32,10 +40,53 @@ public static String bytesToString(byte[] byteArray) { return scaledNumeric.toPlainString(); } + // Convert an Avro-encoded PG_NUMERIC byte array to an readable PG_NUMERIC string value. + public static String pgBytesToString(byte[] byteArray) { + try { + DataInputStream dis = new DataInputStream(new ByteArrayInputStream(byteArray)); + byte specialValue = dis.readByte(); + if (specialValue == (byte) 0x01) { + return "NaN"; + } + int scale = dis.readInt(); + byte[] unscaledNumeric = new byte[byteArray.length - 5]; + dis.readFully(unscaledNumeric); + BigDecimal scaledNumeric = new BigDecimal(new BigInteger(unscaledNumeric), scale); + return scaledNumeric.toPlainString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + // Convert a readable NUMERIC string value to an Avro-encoded NUMERIC byte array. public static byte[] stringToBytes(String numeric) { BigDecimal scaledNumeric = new BigDecimal(numeric, new MathContext(PRECISION)).setScale(SCALE); BigInteger unscaledNumeric = scaledNumeric.unscaledValue(); return unscaledNumeric.toByteArray(); } + + // Convert a readable PG_NUMERIC string value to an Avro-encoded PG_NUMERIC byte array. + public static byte[] pgStringToBytes(String numeric) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(bos); + // Use a byte to hold special cases like NaN + if (numeric.equals("NaN")) { + dos.writeByte(1); + dos.flush(); + return bos.toByteArray(); + } else { + dos.writeByte(0); + } + BigDecimal scaledNumeric = new BigDecimal(numeric); + BigInteger unscaledNumeric = scaledNumeric.unscaledValue(); + int scale = scaledNumeric.scale(); + dos.writeInt(scale); + dos.write(unscaledNumeric.toByteArray()); + dos.flush(); + return bos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/common/Type.java b/src/main/java/com/google/cloud/teleport/spanner/common/Type.java index c1debf3fb3..4da85725e4 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/common/Type.java +++ b/src/main/java/com/google/cloud/teleport/spanner/common/Type.java @@ -15,15 +15,11 @@ */ package com.google.cloud.teleport.spanner.common; -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.cloud.teleport.spanner.TypeCode; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.io.Serializable; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -59,6 +55,26 @@ public final class Type implements Serializable { private static final Type TYPE_ARRAY_TIMESTAMP = new Type(Code.ARRAY, TYPE_TIMESTAMP, null); private static final Type TYPE_ARRAY_DATE = new Type(Code.ARRAY, TYPE_DATE, null); + private static final Type TYPE_PG_BOOL = new Type(Code.PG_BOOL, null, null); + private static final Type TYPE_PG_INT8 = new Type(Code.PG_INT8, null, null); + private static final Type TYPE_PG_FLOAT8 = new Type(Code.PG_FLOAT8, null, null); + private static final Type TYPE_PG_VARCHAR = new Type(Code.PG_VARCHAR, null, null); + private static final Type TYPE_PG_TEXT = new Type(Code.PG_TEXT, null, null); + private static final Type TYPE_PG_NUMERIC = new Type(Code.PG_NUMERIC, null, null); + private static final Type TYPE_PG_BYTEA = new Type(Code.PG_BYTEA, null, null); + private static final Type TYPE_PG_TIMESTAMPTZ = new Type(Code.PG_TIMESTAMPTZ, null, null); + private static final Type TYPE_PG_DATE = new Type(Code.PG_DATE, null, null); + private static final Type TYPE_PG_ARRAY_BOOL = new Type(Code.PG_ARRAY, TYPE_PG_BOOL, null); + private static final Type TYPE_PG_ARRAY_INT8 = new Type(Code.PG_ARRAY, TYPE_PG_INT8, null); + private static final Type TYPE_PG_ARRAY_FLOAT8 = new Type(Code.PG_ARRAY, TYPE_PG_FLOAT8, null); + private static final Type TYPE_PG_ARRAY_VARCHAR = new Type(Code.PG_ARRAY, TYPE_PG_VARCHAR, null); + private static final Type TYPE_PG_ARRAY_TEXT = new Type(Code.PG_ARRAY, TYPE_PG_TEXT, null); + private static final Type TYPE_PG_ARRAY_NUMERIC = new Type(Code.PG_ARRAY, TYPE_PG_NUMERIC, null); + private static final Type TYPE_PG_ARRAY_BYTEA = new Type(Code.PG_ARRAY, TYPE_PG_BYTEA, null); + private static final Type TYPE_PG_ARRAY_TIMESTAMPTZ = + new Type(Code.PG_ARRAY, TYPE_PG_TIMESTAMPTZ, null); + private static final Type TYPE_PG_ARRAY_DATE = new Type(Code.PG_ARRAY, TYPE_PG_DATE, null); + private static final int AMBIGUOUS_FIELD = -1; private static final long serialVersionUID = -3076152125004114582L; @@ -121,6 +137,42 @@ public static Type date() { return TYPE_DATE; } + public static Type pgBool() { + return TYPE_PG_BOOL; + } + + public static Type pgInt8() { + return TYPE_PG_INT8; + } + + public static Type pgFloat8() { + return TYPE_PG_FLOAT8; + } + + public static Type pgText() { + return TYPE_PG_TEXT; + } + + public static Type pgVarchar() { + return TYPE_PG_VARCHAR; + } + + public static Type pgNumeric() { + return TYPE_PG_NUMERIC; + } + + public static Type pgBytea() { + return TYPE_PG_BYTEA; + } + + public static Type pgTimestamptz() { + return TYPE_PG_TIMESTAMPTZ; + } + + public static Type pgDate() { + return TYPE_PG_DATE; + } + /** Returns a descriptor for an array of {@code elementType}. */ public static Type array(Type elementType) { Preconditions.checkNotNull(elementType); @@ -148,6 +200,33 @@ public static Type array(Type elementType) { } } + /** Returns a descriptor for an array of PG {@code elementType}. */ + public static Type pgArray(Type elementType) { + Preconditions.checkNotNull(elementType); + switch (elementType.getCode()) { + case PG_BOOL: + return TYPE_PG_ARRAY_BOOL; + case PG_INT8: + return TYPE_PG_ARRAY_INT8; + case PG_FLOAT8: + return TYPE_PG_ARRAY_FLOAT8; + case PG_NUMERIC: + return TYPE_PG_ARRAY_NUMERIC; + case PG_VARCHAR: + return TYPE_PG_ARRAY_VARCHAR; + case PG_TEXT: + return TYPE_PG_ARRAY_TEXT; + case PG_BYTEA: + return TYPE_PG_ARRAY_BYTEA; + case PG_TIMESTAMPTZ: + return TYPE_PG_ARRAY_TIMESTAMPTZ; + case PG_DATE: + return TYPE_PG_ARRAY_DATE; + default: + return new Type(Code.PG_ARRAY, elementType, null); + } + } + /** * Returns a descriptor for a {@code STRUCT} type: an ordered collection of named and typed * fields. @@ -185,42 +264,42 @@ private Type( /** Enumerates the categories of types. */ public enum Code { - BOOL(TypeCode.BOOL), - INT64(TypeCode.INT64), - NUMERIC(TypeCode.NUMERIC), - FLOAT64(TypeCode.FLOAT64), - STRING(TypeCode.STRING), - JSON(TypeCode.JSON), - BYTES(TypeCode.BYTES), - TIMESTAMP(TypeCode.TIMESTAMP), - DATE(TypeCode.DATE), - ARRAY(TypeCode.ARRAY), - STRUCT(TypeCode.STRUCT); - - private static final Map protoToCode; - - static { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (Code code : Code.values()) { - builder.put(code.protoCode(), code); - } - protoToCode = builder.build(); - } + BOOL("BOOL", Dialect.GOOGLE_STANDARD_SQL), + INT64("INT64", Dialect.GOOGLE_STANDARD_SQL), + NUMERIC("NUMERIC", Dialect.GOOGLE_STANDARD_SQL), + FLOAT64("FLOAT64", Dialect.GOOGLE_STANDARD_SQL), + STRING("STRING", Dialect.GOOGLE_STANDARD_SQL), + JSON("JSON", Dialect.GOOGLE_STANDARD_SQL), + BYTES("BYTES", Dialect.GOOGLE_STANDARD_SQL), + TIMESTAMP("TIMESTAMP", Dialect.GOOGLE_STANDARD_SQL), + DATE("DATE", Dialect.GOOGLE_STANDARD_SQL), + ARRAY("ARRAY", Dialect.GOOGLE_STANDARD_SQL), + STRUCT("STRUCT", Dialect.GOOGLE_STANDARD_SQL), + PG_BOOL("boolean", Dialect.POSTGRESQL), + PG_INT8("bigint", Dialect.POSTGRESQL), + PG_FLOAT8("double precision", Dialect.POSTGRESQL), + PG_TEXT("text", Dialect.POSTGRESQL), + PG_VARCHAR("character varying", Dialect.POSTGRESQL), + PG_NUMERIC("numeric", Dialect.POSTGRESQL), + PG_BYTEA("bytea", Dialect.POSTGRESQL), + PG_TIMESTAMPTZ("timestamp with time zone", Dialect.POSTGRESQL), + PG_DATE("date", Dialect.POSTGRESQL), + PG_ARRAY("array", Dialect.POSTGRESQL); - private final TypeCode protoCode; + private final String name; + private final Dialect dialect; - Code(TypeCode protoCode) { - this.protoCode = protoCode; + Code(String name, Dialect dialect) { + this.name = name; + this.dialect = dialect; } - TypeCode protoCode() { - return protoCode; + public String getName() { + return name; } - static Code fromProtoCode(TypeCode protoCode) { - Code code = protoToCode.get(protoCode); - checkArgument(code != null, "Invalid code: %s", protoCode); - return code; + public Dialect getDialect() { + return dialect; } } @@ -277,7 +356,8 @@ public Code getCode() { * @throws IllegalStateException if {@code code() != Code.ARRAY} */ public Type getArrayElementType() { - Preconditions.checkState(code == Code.ARRAY, "Illegal call for non-ARRAY type"); + Preconditions.checkState( + code == Code.ARRAY || code == Code.PG_ARRAY, "Illegal call for non-ARRAY type"); return arrayElementType; } @@ -334,6 +414,9 @@ void toString(StringBuilder b) { b.append("ARRAY<"); arrayElementType.toString(b); b.append('>'); + } else if (code == Code.PG_ARRAY) { + arrayElementType.toString(b); + b.append("[]"); } else if (code == Code.STRUCT) { b.append("STRUCT<"); for (int i = 0; i < structFields.size(); ++i) { @@ -375,73 +458,4 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(code, arrayElementType, structFields); } - - com.google.cloud.teleport.spanner.Type toProto() { - com.google.cloud.teleport.spanner.Type.Builder proto = - com.google.cloud.teleport.spanner.Type.newBuilder(); - proto.setCode(code.protoCode()); - if (code == Code.ARRAY) { - proto.setArrayElementType(arrayElementType.toProto()); - } else if (code == Code.STRUCT) { - com.google.cloud.teleport.spanner.StructType.Builder fields = proto.getStructTypeBuilder(); - for (StructField field : structFields) { - fields.addFieldsBuilder().setName(field.getName()).setType(field.getType().toProto()); - } - } - return proto.build(); - } - - static com.google.cloud.teleport.spanner.common.Type fromProto( - com.google.cloud.teleport.spanner.Type proto) { - Code type = Code.fromProtoCode(proto.getCode()); - switch (type) { - case BOOL: - return bool(); - case INT64: - return int64(); - case FLOAT64: - return float64(); - case NUMERIC: - return numeric(); - case STRING: - return string(); - case JSON: - return json(); - case BYTES: - return bytes(); - case TIMESTAMP: - return timestamp(); - case DATE: - return date(); - case ARRAY: - checkArgument( - proto.hasArrayElementType(), - "Missing expected 'array_element_type' field in 'Type' message: %s", - proto); - Type elementType; - try { - elementType = fromProto(proto.getArrayElementType()); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException( - "Could not parse 'array_element_type' attribute in 'Type' message: " + proto, e); - } - return array(elementType); - case STRUCT: - checkArgument( - proto.hasStructType(), - "Missing expected 'struct_type' field in 'Type' message: %s", - proto); - List fields = new ArrayList<>(proto.getStructType().getFieldsCount()); - for (com.google.cloud.teleport.spanner.StructType.Field field : - proto.getStructType().getFieldsList()) { - checkArgument(field.hasType(), "Missing expected 'type' attribute in 'Field': %s", proto); - // Names may be empty; for example, the name of the column returned by "SELECT 1". - String name = Strings.nullToEmpty(field.getName()); - fields.add(StructField.of(name, fromProto(field.getType()))); - } - return struct(fields); - default: - throw new AssertionError("Unimplemented case: " + type); - } - } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java index 942d214278..4658240274 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java @@ -47,6 +47,9 @@ public abstract class Column implements Serializable { public abstract Dialect dialect(); + @Nullable + public abstract String defaultExpression(); + public static Builder builder(Dialect dialect) { return new AutoValue_Column.Builder() .dialect(dialect) @@ -62,11 +65,23 @@ public static Builder builder() { } public void prettyPrint(Appendable appendable) throws IOException { - appendable.append(String.format("%1$-40s", "`" + name() + "`")).append(typeString()); + if (dialect() != Dialect.GOOGLE_STANDARD_SQL && dialect() != Dialect.POSTGRESQL) { + throw new IllegalArgumentException(String.format("Unrecognized Dialect: %s.", dialect())); + } + String identifierQuote = DdlUtilityComponents.identifierQuote(dialect()); + appendable + .append(String.format("%1$-40s", identifierQuote + name() + identifierQuote)) + .append(typeString()); if (notNull()) { appendable.append(" NOT NULL"); } + if ((dialect() == Dialect.POSTGRESQL) && defaultExpression() != null) { + appendable.append(" DEFAULT ").append(defaultExpression()); + } if (isGenerated()) { + if (dialect() == Dialect.POSTGRESQL) { + appendable.append(" GENERATED ALWAYS"); + } appendable.append(" AS (").append(generationExpression()).append(")"); if (isStored()) { appendable.append(" STORED"); @@ -106,25 +121,50 @@ private static String typeString(Type type, Integer size) { switch (type.getCode()) { case BOOL: return "BOOL"; + case PG_BOOL: + return "boolean"; case INT64: return "INT64"; + case PG_INT8: + return "bigint"; case FLOAT64: return "FLOAT64"; + case PG_FLOAT8: + return "double precision"; case STRING: return "STRING(" + (size == -1 ? "MAX" : Integer.toString(size)) + ")"; + case PG_VARCHAR: + return "character varying" + (size == -1 ? "" : ("(" + Integer.toString(size) + ")")); + case PG_TEXT: + return "text"; case BYTES: return "BYTES(" + (size == -1 ? "MAX" : Integer.toString(size)) + ")"; + case PG_BYTEA: + return "bytea"; case DATE: return "DATE"; + case PG_DATE: + return "date"; case TIMESTAMP: return "TIMESTAMP"; + case PG_TIMESTAMPTZ: + return "timestamp with time zone"; case NUMERIC: return "NUMERIC"; + case PG_NUMERIC: + return "numeric"; case JSON: return "JSON"; case ARRAY: - Type arrayType = type.getArrayElementType(); - return "ARRAY<" + typeString(arrayType, size) + ">"; + { + Type arrayType = type.getArrayElementType(); + return "ARRAY<" + typeString(arrayType, size) + ">"; + } + case PG_ARRAY: + { + Type arrayType = type.getArrayElementType(); + return typeString(arrayType, size) + "[]"; + } } throw new IllegalArgumentException("Unknown type " + type); @@ -151,6 +191,8 @@ Builder tableBuilder(Table.Builder tableBuilder) { abstract Builder dialect(Dialect dialect); + abstract Dialect dialect(); + public Builder notNull() { return notNull(true); } @@ -159,6 +201,8 @@ public Builder notNull() { public abstract Builder generationExpression(String expression); + public abstract Builder defaultExpression(String expression); + public Builder generatedAs(String expression) { return isGenerated(true).generationExpression(expression); } @@ -175,35 +219,70 @@ public Builder int64() { return type(Type.int64()); } - public Builder float64() { + public Builder pgInt8() { + return type(Type.pgInt8()); + } + public Builder float64() { return type(Type.float64()); } + public Builder pgFloat8() { + return type(Type.pgFloat8()); + } + public Builder bool() { return type(Type.bool()); } + public Builder pgBool() { + return type(Type.pgBool()); + } + public Builder string() { return type(Type.string()); } + public Builder pgVarchar() { + return type(Type.pgVarchar()).max(); + } + + public Builder pgText() { + return type(Type.pgText()).max(); + } + public Builder bytes() { return type(Type.bytes()); } + public Builder pgBytea() { + return type(Type.pgBytea()).max(); + } + public Builder timestamp() { return type(Type.timestamp()); } + public Builder pgTimestamptz() { + return type(Type.pgTimestamptz()); + } + public Builder date() { return type(Type.date()); } + public Builder pgDate() { + return type(Type.pgDate()); + } + public Builder numeric() { return type(Type.numeric()); } + public Builder pgNumeric() { + return type(Type.pgNumeric()); + } + public Builder json() { return type(Type.json()); } @@ -213,7 +292,7 @@ public Builder max() { } public Builder parseType(String spannerType) { - SizedType sizedType = parseSpannerType(spannerType); + SizedType sizedType = parseSpannerType(spannerType, dialect()); return type(sizedType.type).size(sizedType.size); } @@ -240,43 +319,94 @@ private static SizedType t(Type type, Integer size) { return new SizedType(type, size); } - private static SizedType parseSpannerType(String spannerType) { - if (spannerType.equals("BOOL")) { - return t(Type.bool(), null); - } - if (spannerType.equals("INT64")) { - return t(Type.int64(), null); - } - if (spannerType.equals("FLOAT64")) { - return t(Type.float64(), null); - } - if (spannerType.startsWith("STRING")) { - String sizeStr = spannerType.substring(7, spannerType.length() - 1); - int size = sizeStr.equals("MAX") ? -1 : Integer.parseInt(sizeStr); - return t(Type.string(), size); - } - if (spannerType.startsWith("BYTES")) { - String sizeStr = spannerType.substring(6, spannerType.length() - 1); - int size = sizeStr.equals("MAX") ? -1 : Integer.parseInt(sizeStr); - return t(Type.bytes(), size); - } - if (spannerType.equals("TIMESTAMP")) { - return t(Type.timestamp(), null); - } - if (spannerType.equals("DATE")) { - return t(Type.date(), null); - } - if (spannerType.equals("NUMERIC")) { - return t(Type.numeric(), null); - } - if (spannerType.equals("JSON")) { - return t(Type.json(), null); - } - if (spannerType.startsWith("ARRAY")) { - // Substring "ARRAY<"xxx">" - String spannerArrayType = spannerType.substring(6, spannerType.length() - 1); - SizedType itemType = parseSpannerType(spannerArrayType); - return t(Type.array(itemType.type), itemType.size); + private static SizedType parseSpannerType(String spannerType, Dialect dialect) { + switch (dialect) { + case GOOGLE_STANDARD_SQL: + { + if (spannerType.equals("BOOL")) { + return t(Type.bool(), null); + } + if (spannerType.equals("INT64")) { + return t(Type.int64(), null); + } + if (spannerType.equals("FLOAT64")) { + return t(Type.float64(), null); + } + if (spannerType.startsWith("STRING")) { + String sizeStr = spannerType.substring(7, spannerType.length() - 1); + int size = sizeStr.equals("MAX") ? -1 : Integer.parseInt(sizeStr); + return t(Type.string(), size); + } + if (spannerType.startsWith("BYTES")) { + String sizeStr = spannerType.substring(6, spannerType.length() - 1); + int size = sizeStr.equals("MAX") ? -1 : Integer.parseInt(sizeStr); + return t(Type.bytes(), size); + } + if (spannerType.equals("TIMESTAMP")) { + return t(Type.timestamp(), null); + } + if (spannerType.equals("DATE")) { + return t(Type.date(), null); + } + if (spannerType.equals("NUMERIC")) { + return t(Type.numeric(), null); + } + if (spannerType.equals("JSON")) { + return t(Type.json(), null); + } + if (spannerType.startsWith("ARRAY")) { + // Substring "ARRAY" + String spannerArrayType = spannerType.substring(6, spannerType.length() - 1); + SizedType itemType = parseSpannerType(spannerArrayType, dialect); + return t(Type.array(itemType.type), itemType.size); + } + break; + } + case POSTGRESQL: + { + if (spannerType.endsWith("[]")) { + // Substring "xxx[]" + // Must check array type first + String spannerArrayType = spannerType.substring(0, spannerType.length() - 2); + SizedType itemType = parseSpannerType(spannerArrayType, dialect); + return t(Type.pgArray(itemType.type), itemType.size); + } + if (spannerType.equals("boolean")) { + return t(Type.pgBool(), null); + } + if (spannerType.equals("bigint")) { + return t(Type.pgInt8(), null); + } + if (spannerType.equals("double precision")) { + return t(Type.pgFloat8(), null); + } + if (spannerType.equals("text")) { + return t(Type.pgText(), -1); + } + if (spannerType.startsWith("character varying")) { + int size = -1; + if (spannerType.length() > 18) { + String sizeStr = spannerType.substring(18, spannerType.length() - 1); + size = Integer.parseInt(sizeStr); + } + return t(Type.pgVarchar(), size); + } + if (spannerType.equals("bytea")) { + return t(Type.pgBytea(), -1); + } + if (spannerType.equals("timestamp with time zone")) { + return t(Type.pgTimestamptz(), null); + } + if (spannerType.equals("numeric")) { + return t(Type.pgNumeric(), null); + } + if (spannerType.equals("date")) { + return t(Type.pgDate(), null); + } + break; + } + default: + break; } throw new IllegalArgumentException("Unknown spanner type " + spannerType); } diff --git a/src/main/proto/type.proto b/src/main/proto/type.proto deleted file mode 100644 index 0b82c4b6f6..0000000000 --- a/src/main/proto/type.proto +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright 2020 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. - -syntax = "proto3"; - -package com.google.cloud.teleport.spanner; - -option java_multiple_files = true; -option java_outer_classname = "TypeProto"; -option java_package = "com.google.cloud.teleport.spanner"; - -// `Type` indicates the type of a Cloud Spanner value, as might be stored in a -// table cell or returned from an SQL query. -message Type { - // Required. The [TypeCode][google.spanner.v1.TypeCode] for this type. - TypeCode code = 1; - - // If [code][google.spanner.v1.Type.code] == - // [ARRAY][google.spanner.v1.TypeCode.ARRAY], then `array_element_type` is the - // type of the array elements. - Type array_element_type = 2; - - // If [code][google.spanner.v1.Type.code] == - // [STRUCT][google.spanner.v1.TypeCode.STRUCT], then `struct_type` provides - // type information for the struct's fields. - StructType struct_type = 3; -} - -// `StructType` defines the fields of a -// [STRUCT][google.spanner.v1.TypeCode.STRUCT] type. -message StructType { - // Message representing a single field of a struct. - message Field { - // The name of the field. For reads, this is the column name. For - // SQL queries, it is the column alias (e.g., `"Word"` in the - // query `"SELECT 'hello' AS Word"`), or the column name (e.g., - // `"ColName"` in the query `"SELECT ColName FROM Table"`). Some - // columns might have an empty name (e.g., !"SELECT - // UPPER(ColName)"`). Note that a query result can contain - // multiple fields with the same name. - string name = 1; - - // The type of the field. - Type type = 2; - } - - // The list of fields that make up this struct. Order is - // significant, because values of this struct type are represented as - // lists, where the order of field values matches the order of - // fields in the [StructType][google.spanner.v1.StructType]. In turn, the - // order of fields matches the order of columns in a read request, or the - // order of fields in the `SELECT` clause of a query. - repeated Field fields = 1; -} - -// `TypeCode` is used as part of [Type][google.spanner.v1.Type] to -// indicate the type of a Cloud Spanner value. -// -// Each legal value of a type can be encoded to or decoded from a JSON -// value, using the encodings described below. All Cloud Spanner values can -// be `null`, regardless of type; `null`s are always encoded as a JSON -// `null`. -enum TypeCode { - // Not specified. - TYPE_CODE_UNSPECIFIED = 0; - - // Encoded as JSON `true` or `false`. - BOOL = 1; - - // Encoded as `string`, in decimal format. - INT64 = 2; - - // Encoded as `number`, or the strings `"NaN"`, `"Infinity"`, or - // `"-Infinity"`. - FLOAT64 = 3; - - // Encoded as `string` in RFC 3339 timestamp format. The time zone - // must be present, and must be `"Z"`. - // - // If the schema has the column option - // `allow_commit_timestamp=true`, the placeholder string - // `"spanner.commit_timestamp()"` can be used to instruct the system - // to insert the commit timestamp associated with the transaction - // commit. - TIMESTAMP = 4; - - // Encoded as `string` in RFC 3339 date format. - DATE = 5; - - // Encoded as `string`. - STRING = 6; - - // Encoded as a base64-encoded `string`, as described in RFC 4648, - // section 4. - BYTES = 7; - - // Encoded as `list`, where the list elements are represented - // according to - // [array_element_type][google.spanner.v1.Type.array_element_type]. - ARRAY = 8; - - // Encoded as `list`, where list element `i` is represented according - // to [struct_type.fields[i]][google.spanner.v1.StructType.fields]. - STRUCT = 9; - - // Encoded as `string`, in decimal format or scientific notation format. - //
Decimal format: - //
`[+-]Digits[.[Digits]]` or - //
`[+-][Digits].Digits` - // - // Scientific notation: - //
`[+-]Digits[.[Digits]][ExponentIndicator[+-]Digits]` or - //
`[+-][Digits].Digits[ExponentIndicator[+-]Digits]` - //
(ExponentIndicator is `"e"` or `"E"`) - NUMERIC = 10; - - // Encoded as a JSON-formatted 'string' as described in RFC 7159. The - // following rules will be applied when parsing JSON input: - // - Whitespace will be stripped from the document. - // - If a JSON object has duplicate keys, only the first key will be - // preserved. - // - Members of a JSON object are not guaranteed to have their order - // preserved. JSON array elements will have their order preserved. - JSON = 11; -} diff --git a/src/test/java/com/google/cloud/teleport/spanner/common/NumericUtilsTest.java b/src/test/java/com/google/cloud/teleport/spanner/common/NumericUtilsTest.java new file mode 100644 index 0000000000..079890da91 --- /dev/null +++ b/src/test/java/com/google/cloud/teleport/spanner/common/NumericUtilsTest.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2022 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.common; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.core.IsNot.not; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +/** Test coverage for {@link NumericUtils}. */ +public class NumericUtilsTest { + + @Test + public void testExactScale() { + String numericValue = "42.42"; + byte[] encodedBytes = NumericUtils.pgStringToBytes(numericValue); + assertThat(numericValue, equalTo(NumericUtils.pgBytesToString(encodedBytes))); + + String numericValueWithHighScale = "42.42000000000000000000000"; + byte[] encodedHighScaleBytes = NumericUtils.pgStringToBytes(numericValueWithHighScale); + assertThat( + numericValueWithHighScale, equalTo(NumericUtils.pgBytesToString(encodedHighScaleBytes))); + + assertThat(encodedBytes, not(equalTo(encodedHighScaleBytes))); + assertTrue(encodedBytes.length < encodedHighScaleBytes.length); + } +} diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index adace3b764..258e5bb304 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -112,19 +112,21 @@ public void pgSimple() { builder .createTable("Users") .column("id") - .int64() + .pgInt8() .notNull() .endColumn() .column("first_name") - .string() + .pgVarchar() .size(10) + .defaultExpression("John") .endColumn() .column("last_name") - .type(Type.string()) + .type(Type.pgVarchar()) .max() + .defaultExpression("Lennon") .endColumn() .column("full_name") - .type(Type.string()) + .type(Type.pgVarchar()) .max() .generatedAs("CONCAT(first_name, ' ', last_name)") .stored() @@ -156,10 +158,11 @@ public void pgSimple() { equalToCompressingWhiteSpace( "ALTER DATABASE `%db_name%` SET OPTIONS ( version_retention_period = 4d )" + " CREATE TABLE \"Users\" (" - + " `id` INT64 NOT NULL," - + " `first_name` STRING(10)," - + " `last_name` STRING(MAX)," - + " `full_name` STRING(MAX) AS (CONCAT(first_name, ' ', last_name)) STORED," + + " \"id\" bigint NOT NULL," + + " \"first_name\" character varying(10) DEFAULT John," + + " \"last_name\" character varying DEFAULT Lennon," + + " \"full_name\" character varying GENERATED ALWAYS AS" + + " (CONCAT(first_name, ' ', last_name)) STORED," + " CONSTRAINT \"ck\" CHECK (\"first_name\" != \"last_name\")," + " PRIMARY KEY (\"id\")" + " ) " @@ -231,15 +234,15 @@ public void pgInterleaves() { Ddl.builder(Dialect.POSTGRESQL) .createTable("Users") .column("id") - .int64() + .pgInt8() .notNull() .endColumn() .column("first_name") - .string() + .pgVarchar() .size(10) .endColumn() .column("last_name") - .type(Type.string()) + .type(Type.pgVarchar()) .max() .endColumn() .primaryKey() @@ -248,15 +251,15 @@ public void pgInterleaves() { .endTable() .createTable("Account") .column("id") - .int64() + .pgInt8() .notNull() .endColumn() .column("balanceId") - .int64() + .pgInt8() .notNull() .endColumn() .column("balance") - .float64() + .pgFloat8() .notNull() .endColumn() .primaryKey() @@ -270,15 +273,15 @@ public void pgInterleaves() { ddl.prettyPrint(), equalToCompressingWhiteSpace( "CREATE TABLE \"Users\" (" - + " `id` INT64 NOT NULL," - + " `first_name` STRING(10)," - + " `last_name` STRING(MAX)," + + " \"id\" bigint NOT NULL," + + " \"first_name\" character varying(10)," + + " \"last_name\" character varying," + " PRIMARY KEY (\"id\")" + " ) " + " CREATE TABLE \"Account\" (" - + " `id` INT64 NOT NULL," - + " `balanceId` INT64 NOT NULL," - + " `balance` FLOAT64 NOT NULL," + + " \"id\" bigint NOT NULL," + + " \"balanceId\" bigint NOT NULL," + + " \"balance\" double precision NOT NULL," + " PRIMARY KEY (\"id\")" + " ) " + " INTERLEAVE IN PARENT \"Users\" ON DELETE CASCADE")); From f7642fd96518d1f670c65f00bda527d0ab1d99f7 Mon Sep 17 00:00:00 2001 From: zhoufek Date: Tue, 10 May 2022 06:40:40 -0700 Subject: [PATCH 127/145] Add instructions for using Maven to README PiperOrigin-RevId: 447719292 --- README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.md b/README.md index 802f4fbf4f..fed96ed557 100644 --- a/README.md +++ b/README.md @@ -19,6 +19,14 @@ As of November 18, 2021, our default branch is now named "main". This does not affect forks. If you would like your fork and its local clone to reflect these changes you can follow [GitHub's branch renaming guide](https://docs.github.com/en/repositories/configuring-branches-and-merges-in-your-repository/managing-branches-in-your-repository/renaming-a-branch). +## Building + +Maven commands should be run on the `unified-templates.xml` aggregator POM. An example would be: + +``` +mvn clean install -f unified-templates.xml -pl v2/pubsub-binary-to-bigquery -am +``` + ## Template Pipelines * [BigQuery to Bigtable](v2/bigquery-to-bigtable/src/main/java/com/google/cloud/teleport/v2/templates/BigQueryToBigtable.java) From 25d3f5fd8a281679f81663714f68eaa4fb97a68e Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 12:08:37 -0700 Subject: [PATCH 128/145] Add new attributes to support PG in Index and IndexColumn DDL. PiperOrigin-RevId: 447795055 --- .../cloud/teleport/spanner/ddl/Index.java | 59 +++++++- .../teleport/spanner/ddl/IndexColumn.java | 127 ++++++++++++++++-- .../cloud/teleport/spanner/ddl/DdlTest.java | 72 ++++++++++ 3 files changed, 249 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java index 4dfda721ca..7c8ea8729f 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java @@ -38,13 +38,21 @@ public abstract class Index implements Serializable { abstract boolean unique(); + // restricted for gsql abstract boolean nullFiltered(); + // restricted for pg + @Nullable + abstract String filter(); + @Nullable abstract String interleaveIn(); public static Builder builder(Dialect dialect) { - return new AutoValue_Index.Builder().dialect(dialect).nullFiltered(false).unique(false); + return new AutoValue_Index.Builder() + .dialect(dialect) + .nullFiltered(false) + .unique(false); } public static Builder builder() { @@ -52,6 +60,52 @@ public static Builder builder() { } public void prettyPrint(Appendable appendable) throws IOException { + switch (dialect()) { + case GOOGLE_STANDARD_SQL: + prettyPrintGsql(appendable); + break; + case POSTGRESQL: + prettyPrintPg(appendable); + break; + default: + throw new IllegalArgumentException(String.format("Unrecognized dialect: ", dialect())); + } + } + + private void prettyPrintPg(Appendable appendable) throws IOException { + appendable.append("CREATE"); + if (unique()) { + appendable.append(" UNIQUE"); + } + appendable.append(" INDEX \"").append(name()).append("\" ON \"").append(table()).append("\""); + + String indexColumnsString = + indexColumns().stream() + .filter(c -> c.order() != IndexColumn.Order.STORING) + .map(c -> c.prettyPrint()) + .collect(Collectors.joining(", ")); + appendable.append("(").append(indexColumnsString).append(")"); + + String storingString = + indexColumns().stream() + .filter(c -> c.order() == IndexColumn.Order.STORING) + .map(c -> "\"" + c.name() + "\"") + .collect(Collectors.joining(", ")); + + if (!storingString.isEmpty()) { + appendable.append(" INCLUDE (").append(storingString).append(")"); + } + + if (interleaveIn() != null) { + appendable.append(" INTERLEAVE IN \"").append(interleaveIn()).append("\""); + } + + if (filter() != null && !filter().isEmpty()) { + appendable.append(" WHERE ").append(filter()); + } + } + + private void prettyPrintGsql(Appendable appendable) throws IOException { appendable.append("CREATE"); if (unique()) { appendable.append(" UNIQUE"); @@ -77,6 +131,7 @@ public void prettyPrint(Appendable appendable) throws IOException { if (!storingString.isEmpty()) { appendable.append(" STORING (").append(storingString).append(")"); } + if (interleaveIn() != null) { appendable.append(", INTERLEAVE IN ").append(interleaveIn()); } @@ -139,6 +194,8 @@ public Builder nullFiltered() { return nullFiltered(true); } + public abstract Builder filter(String filter); + public abstract Builder interleaveIn(String interleaveIn); abstract Index autoBuild(); diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java index def1040ec4..bc263046a6 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java @@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.Serializable; +import javax.annotation.Nullable; /** A Cloud Spanner index column. */ @AutoValue @@ -32,12 +33,20 @@ public abstract class IndexColumn implements Serializable { public abstract Dialect dialect(); + // restricted to PG + @Nullable + public abstract NullsOrder nullsOrder(); + public static IndexColumn create(String name, Order order, Dialect dialect) { - return new AutoValue_IndexColumn(name, order, dialect); + return new AutoValue_IndexColumn.Builder() + .dialect(dialect) + .name(name) + .order(order) + .autoBuild(); } public static IndexColumn create(String name, Order order) { - return new AutoValue_IndexColumn(name, order, Dialect.GOOGLE_STANDARD_SQL); + return create(name, order, Dialect.GOOGLE_STANDARD_SQL); } /** Ordering of column in the index. */ @@ -57,8 +66,28 @@ public static Order defaultOrder() { private final String title; } + /** Ordering of null values in the column. */ + public enum NullsOrder { + FIRST("FIRST"), + LAST("LAST"); + + NullsOrder(String title) { + this.title = title; + } + + private final String title; + } + public void prettyPrint(Appendable appendable) throws IOException { - appendable.append("`").append(name()).append("` ").append(order().title); + String identifierQuote = DdlUtilityComponents.identifierQuote(dialect()); + appendable + .append(identifierQuote) + .append(name()) + .append(identifierQuote + " ") + .append(order().title); + if (nullsOrder() != null) { + appendable.append(" NULLS ").append(nullsOrder().title); + } } public String prettyPrint() { @@ -77,9 +106,26 @@ public String toString() { } /** A builder for {@link IndexColumn}. */ + @AutoValue.Builder + public abstract static class Builder { + + abstract Builder name(String name); + + abstract Builder order(Order order); + + abstract Builder dialect(Dialect dialect); + + abstract Builder nullsOrder(NullsOrder nullsOrder); + + abstract IndexColumn autoBuild(); + } + + /** A builder for {@link IndexColumns}. */ public static class IndexColumnsBuilder { private ImmutableList.Builder columns = ImmutableList.builder(); + private Builder indexColumnBuilder; + private T callback; private Dialect dialect; @@ -94,11 +140,6 @@ public IndexColumnsBuilder asc(String name) { return set(indexColumn); } - public IndexColumnsBuilder set(IndexColumn indexColumn) { - columns.add(indexColumn); - return this; - } - public IndexColumnsBuilder desc(String name) { return set(IndexColumn.create(name, Order.DESC, dialect)); } @@ -107,10 +148,80 @@ public IndexColumnsBuilder storing(String name) { return set(IndexColumn.create(name, Order.STORING, dialect)); } + IndexColumnsBuilder set(IndexColumn indexColumn) { + columns.add(indexColumn); + return this; + } + + public IndexColumnsBuilder create() { + indexColumnBuilder = new AutoValue_IndexColumn.Builder().dialect(dialect); + return this; + } + + public IndexColumnsBuilder name(String name) { + if (indexColumnBuilder == null) { + throw new IllegalArgumentException( + "Builder is missing. Call create method to initiate a builder first."); + } + indexColumnBuilder.name(name); + return this; + } + + public IndexColumnsBuilder asc() { + if (indexColumnBuilder == null) { + throw new IllegalArgumentException( + "Builder is missing. Call create method to initiate a builder first."); + } + indexColumnBuilder.order(Order.ASC); + return this; + } + + public IndexColumnsBuilder desc() { + if (indexColumnBuilder == null) { + throw new IllegalArgumentException( + "Builder is missing. Call create method to initiate a builder first."); + } + indexColumnBuilder.order(Order.DESC); + return this; + } + + public IndexColumnsBuilder storing() { + if (indexColumnBuilder == null) { + throw new IllegalArgumentException( + "Builder is missing. Call create method to initiate a builder first."); + } + indexColumnBuilder.order(Order.STORING); + return this; + } + + public IndexColumnsBuilder nullsFirst() { + if (indexColumnBuilder == null) { + throw new IllegalArgumentException( + "Builder is missing. Call create method to initiate a builder first."); + } + indexColumnBuilder.nullsOrder(NullsOrder.FIRST); + return this; + } + + public IndexColumnsBuilder nullsLast() { + if (indexColumnBuilder == null) { + throw new IllegalArgumentException( + "Builder is missing. Call create method to initiate a builder first."); + } + indexColumnBuilder.nullsOrder(NullsOrder.LAST); + return this; + } + public ImmutableList build() { return columns.build(); } + public IndexColumnsBuilder endIndexColumn() { + set(indexColumnBuilder.autoBuild()); + indexColumnBuilder = null; + return this; + } + public T end() { return callback; } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index 258e5bb304..f7a0380227 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -333,6 +333,78 @@ public void pgTestDatabaseOptions() { is("ALTER DATABASE `database_id` SET OPTIONS ( version_retention_period = 4d )")); } + @Test + public void pgTestIndex() { + Index.Builder builder = + Index.builder(Dialect.POSTGRESQL) + .name("user_index") + .table("User") + .unique() + .filter("\"first_name\" IS NOT NULL AND \"last_name\" IS NOT NULL"); + builder + .columns() + .create() + .name("first_name") + .asc() + .endIndexColumn() + .create() + .name("last_name") + .desc() + .endIndexColumn() + .create() + .name("full_name") + .storing() + .endIndexColumn() + .end(); + Index index = builder.build(); + assertThat( + index.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE UNIQUE INDEX \"user_index\" ON \"User\"(\"first_name\" ASC," + + " \"last_name\" DESC) INCLUDE (\"full_name\") WHERE \"first_name\" IS" + + " NOT NULL AND \"last_name\" IS NOT NULL")); + } + + @Test + public void pgTestIndexNullsOrder() { + Index.Builder builder = + Index.builder(Dialect.POSTGRESQL).name("user_index").table("User").unique(); + builder + .columns() + .create() + .name("first_name") + .asc() + .nullsFirst() + .endIndexColumn() + .create() + .name("last_name") + .desc() + .nullsLast() + .endIndexColumn() + .create() + .name("first_nick_name") + .asc() + .nullsLast() + .endIndexColumn() + .create() + .name("last_nick_name") + .desc() + .nullsFirst() + .endIndexColumn() + .create() + .name("full_name") + .storing() + .endIndexColumn() + .end(); + Index index = builder.build(); + assertThat( + index.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE UNIQUE INDEX \"user_index\" ON \"User\"(\"first_name\" ASC NULLS FIRST," + + " \"last_name\" DESC NULLS LAST, \"first_nick_name\" ASC NULLS LAST," + + " \"last_nick_name\" DESC NULLS FIRST) INCLUDE (\"full_name\")")); + } + @Test public void changeStreams() { Ddl ddl = From c429315cf282aeaec71f259cc070df9480e6339f Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 13:28:04 -0700 Subject: [PATCH 129/145] Support for Cloud Spanner Check Constraints, Views and Foreign Keys in PostgreSQL Interface. PiperOrigin-RevId: 447813713 --- .../teleport/spanner/ddl/CheckConstraint.java | 15 ++++-- .../cloud/teleport/spanner/ddl/Ddl.java | 35 +++++++++---- .../spanner/ddl/DdlUtilityComponents.java | 13 +++++ .../teleport/spanner/ddl/ForeignKey.java | 29 ++++++++--- .../cloud/teleport/spanner/ddl/View.java | 13 ++++- .../cloud/teleport/spanner/ddl/DdlTest.java | 52 ++++++++++++++++++- 6 files changed, 131 insertions(+), 26 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java index e10b50f672..2a9570dd1e 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java @@ -28,17 +28,24 @@ public abstract class CheckConstraint implements Serializable { public abstract String expression(); + public abstract Dialect dialect(); + public abstract Builder toBuilder(); + public static Builder builder(Dialect dialect) { + return new AutoValue_CheckConstraint.Builder().dialect(dialect); + } + public static Builder builder() { - return new AutoValue_CheckConstraint.Builder(); + return builder(Dialect.GOOGLE_STANDARD_SQL); } private void prettyPrint(Appendable appendable) throws IOException { + String identifierQuote = DdlUtilityComponents.identifierQuote(dialect()); appendable - .append("CONSTRAINT `") + .append("CONSTRAINT " + identifierQuote) .append(name()) - .append("` CHECK (") + .append(identifierQuote + " CHECK (") .append(expression()) .append(")"); } @@ -65,6 +72,8 @@ public abstract static class Builder { public abstract Builder expression(String expr); + abstract Builder dialect(Dialect dialect); + public abstract CheckConstraint build(); } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java index 8e5bbcfa3b..600d2d4e31 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java @@ -145,7 +145,7 @@ public ImmutableList databaseOptions() { public void prettyPrint(Appendable appendable) throws IOException { for (Export.DatabaseOption databaseOption : databaseOptions()) { - appendable.append(getDatabaseOptionsStatements(databaseOption, "%db_name%")); + appendable.append(getDatabaseOptionsStatements(databaseOption, "%db_name%", dialect)); appendable.append("\n"); } @@ -262,24 +262,37 @@ public List createChangeStreamStatements() { public List setOptionsStatements(String databaseId) { List result = new ArrayList<>(); for (Export.DatabaseOption databaseOption : databaseOptions()) { - result.add(getDatabaseOptionsStatements(databaseOption, databaseId)); + result.add(getDatabaseOptionsStatements(databaseOption, databaseId, dialect)); } return result; } private static String getDatabaseOptionsStatements( - Export.DatabaseOption databaseOption, String databaseId) { + Export.DatabaseOption databaseOption, String databaseId, Dialect dialect) { + String literalQuote = DdlUtilityComponents.literalQuote(dialect); String formattedValue = databaseOption.getOptionType().equalsIgnoreCase("STRING") - ? "\"" + ? literalQuote + DdlUtilityComponents.OPTION_STRING_ESCAPER.escape(databaseOption.getOptionValue()) - + "\"" + + literalQuote : databaseOption.getOptionValue(); - - String statement = - String.format( - "ALTER DATABASE `%s` SET OPTIONS ( %s = %s )", - databaseId, databaseOption.getOptionName(), formattedValue); + String statement; + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = + String.format( + "ALTER DATABASE `%s` SET OPTIONS ( %s = %s )", + databaseId, databaseOption.getOptionName(), formattedValue); + break; + case POSTGRESQL: + statement = + String.format( + "ALTER DATABASE \"%s\" SET spanner.%s = %s", + databaseId, databaseOption.getOptionName(), formattedValue); + break; + default: + throw new IllegalArgumentException(String.format("Unrecognized Dialect: %s", dialect)); + } return statement; } @@ -362,7 +375,7 @@ public Collection
tables() { public View.Builder createView(String name) { View view = views.get(name.toLowerCase()); if (view == null) { - return View.builder().name(name).ddlBuilder(this); + return View.builder(dialect).name(name).ddlBuilder(this); } return view.toBuilder().ddlBuilder(this); } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java index f889764d5a..8827e28dec 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java @@ -35,6 +35,8 @@ private DdlUtilityComponents() {} .build(); static final String POSTGRESQL_IDENTIFIER_QUOTE = "\""; static final String GSQL_IDENTIFIER_QUOTE = "`"; + static final String POSTGRESQL_LITERAL_QUOTE = "'"; + static final String GSQL_LITERAL_QUOTE = "\""; static String identifierQuote(Dialect dialect) { switch (dialect) { @@ -46,4 +48,15 @@ static String identifierQuote(Dialect dialect) { throw new IllegalArgumentException(String.format("Unrecognized dialect: %s", dialect)); } } + + static String literalQuote(Dialect dialect) { + switch (dialect) { + case POSTGRESQL: + return POSTGRESQL_LITERAL_QUOTE; + case GOOGLE_STANDARD_SQL: + return GSQL_LITERAL_QUOTE; + default: + throw new IllegalArgumentException(String.format("Unrecognized dialect: %s", dialect)); + } + } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java index 25ceaa6ec0..dd727a7c17 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java @@ -36,25 +36,36 @@ public abstract class ForeignKey implements Serializable { abstract ImmutableList referencedColumns(); + abstract Dialect dialect(); + + public static Builder builder(Dialect dialect) { + return new AutoValue_ForeignKey.Builder().dialect(dialect); + } + public static Builder builder() { - return new AutoValue_ForeignKey.Builder(); + return builder(Dialect.GOOGLE_STANDARD_SQL); } private void prettyPrint(Appendable appendable) throws IOException { + String identifierQuote = DdlUtilityComponents.identifierQuote(dialect()); String columnsString = - columns().stream().map(c -> "`" + c + "`").collect(Collectors.joining(", ")); + columns().stream() + .map(c -> identifierQuote + c + identifierQuote) + .collect(Collectors.joining(", ")); String referencedColumnsString = - referencedColumns().stream().map(c -> "`" + c + "`").collect(Collectors.joining(", ")); + referencedColumns().stream() + .map(c -> identifierQuote + c + identifierQuote) + .collect(Collectors.joining(", ")); appendable - .append("ALTER TABLE `") + .append("ALTER TABLE " + identifierQuote) .append(table()) - .append("` ADD CONSTRAINT `") + .append(identifierQuote + " ADD CONSTRAINT " + identifierQuote) .append(name()) - .append("` FOREIGN KEY (") + .append(identifierQuote + " FOREIGN KEY (") .append(columnsString) - .append(") REFERENCES `") + .append(") REFERENCES " + identifierQuote) .append(referencedTable()) - .append(("` (")) + .append((identifierQuote + " (")) .append(referencedColumnsString) .append(")"); } @@ -84,6 +95,8 @@ public abstract static class Builder { public abstract Builder referencedTable(String name); + abstract Builder dialect(Dialect dialect); + public abstract ImmutableList.Builder columnsBuilder(); public abstract ImmutableList.Builder referencedColumnsBuilder(); diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java index 5741e4769a..d724ee2778 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java @@ -39,10 +39,13 @@ public enum SqlSecurity { @Nullable public abstract SqlSecurity security(); + public abstract Dialect dialect(); + public abstract Builder toBuilder(); public void prettyPrint(Appendable appendable) throws IOException { - appendable.append("CREATE VIEW `").append(name()).append("`"); + String identifierQuote = DdlUtilityComponents.identifierQuote(dialect()); + appendable.append("CREATE VIEW " + identifierQuote).append(name()).append(identifierQuote); SqlSecurity rights = security(); if (rights != null) { appendable.append(" SQL SECURITY ").append(rights.toString()); @@ -65,8 +68,12 @@ public String toString() { return prettyPrint(); } + public static Builder builder(Dialect dialect) { + return new AutoValue_View.Builder().dialect(dialect); + } + public static Builder builder() { - return new AutoValue_View.Builder(); + return builder(Dialect.GOOGLE_STANDARD_SQL); } /** A builder for {@link View}. */ @@ -91,6 +98,8 @@ public Builder ddlBuilder(Ddl.Builder ddlBuilder) { public abstract SqlSecurity security(); + abstract Builder dialect(Dialect dialect); + public abstract View build(); public Ddl.Builder endView() { diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index f7a0380227..b0602531a5 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -149,6 +149,7 @@ public void pgSimple() { Export.DatabaseOption.newBuilder() .setOptionName("version_retention_period") .setOptionValue("4d") + .setOptionType("STRING") .build()) .build(); builder.mergeDatabaseOptions(export.getDatabaseOptionsList()); @@ -156,7 +157,7 @@ public void pgSimple() { assertThat( ddl.prettyPrint(), equalToCompressingWhiteSpace( - "ALTER DATABASE `%db_name%` SET OPTIONS ( version_retention_period = 4d )" + "ALTER DATABASE \"%db_name%\" SET spanner.version_retention_period = '4d'" + " CREATE TABLE \"Users\" (" + " \"id\" bigint NOT NULL," + " \"first_name\" character varying(10) DEFAULT John," @@ -318,6 +319,8 @@ public void pgTestDatabaseOptions() { Export.DatabaseOption.newBuilder() .setOptionName("version_retention_period") .setOptionValue("4d") + .setOptionValue("4d") + .setOptionType("STRING") .build()); dbOptionList.add( Export.DatabaseOption.newBuilder() @@ -330,7 +333,7 @@ public void pgTestDatabaseOptions() { assertThat(optionStatements.size(), is(1)); assertThat( optionStatements.get(0), - is("ALTER DATABASE `database_id` SET OPTIONS ( version_retention_period = 4d )")); + is("ALTER DATABASE \"database_id\" SET spanner.version_retention_period = '4d'")); } @Test @@ -405,6 +408,51 @@ public void pgTestIndexNullsOrder() { + " \"last_nick_name\" DESC NULLS FIRST) INCLUDE (\"full_name\")")); } + @Test + public void pgTestCheckConstraint() { + CheckConstraint checkConstraint = + CheckConstraint.builder(Dialect.POSTGRESQL) + .name("name_check") + .expression("\"first_name\" != \"last_name\"") + .build(); + assertThat( + checkConstraint.prettyPrint(), + equalToCompressingWhiteSpace( + "CONSTRAINT \"name_check\" CHECK (\"first_name\" != \"last_name\")")); + } + + @Test + public void pgTestForeignKey() { + ForeignKey.Builder builder = + ForeignKey.builder(Dialect.POSTGRESQL) + .name("account_to_user") + .table("Account") + .referencedTable("User"); + builder.columnsBuilder().add("account_id", "owner_name"); + builder.referencedColumnsBuilder().add("user_id", "full_name"); + ForeignKey foreignKey = builder.build(); + assertThat( + foreignKey.prettyPrint(), + equalToCompressingWhiteSpace( + "ALTER TABLE \"Account\" ADD CONSTRAINT \"account_to_user\" FOREIGN KEY" + + " (\"account_id\", \"owner_name\") REFERENCES \"User\" (\"user_id\"," + + " \"full_name\")")); + } + + @Test + public void pgTestView() { + View view = + View.builder(Dialect.POSTGRESQL) + .name("user_view") + .query("SELECT * FROM \"User\"") + .security(View.SqlSecurity.INVOKER) + .build(); + assertThat( + view.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE VIEW \"user_view\" SQL SECURITY INVOKER AS SELECT * FROM \"User\"")); + } + @Test public void changeStreams() { Ddl ddl = From 61c1045b46d566448fb7d7ae3b7961070d777039 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 14:59:10 -0700 Subject: [PATCH 130/145] Support for InformationSchemaScanner in PostgreSQL interface PiperOrigin-RevId: 447835988 --- .../spanner/ddl/InformationSchemaScanner.java | 424 ++++++++++++++---- .../spanner/SpannerServerResource.java | 22 +- .../ddl/InformationSchemaScannerTest.java | 283 ++++++++++++ .../ddl/RandomInsertMutationGenerator.java | 26 +- .../spanner/ddl/RandomValueGenerator.java | 73 ++- 5 files changed, 733 insertions(+), 95 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java index e562171997..c2d5b13075 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java @@ -37,19 +37,30 @@ public class InformationSchemaScanner { private final ReadContext context; + private final Dialect dialect; + public InformationSchemaScanner(ReadContext context) { this.context = context; + this.dialect = Dialect.GOOGLE_STANDARD_SQL; + } + + public InformationSchemaScanner(ReadContext context, Dialect dialect) { + this.context = context; + this.dialect = dialect; } public Ddl scan() { - Ddl.Builder builder = Ddl.builder(); + Ddl.Builder builder = Ddl.builder(dialect); listDatabaseOptions(builder); listTables(builder); listViews(builder); listColumns(builder); listColumnOptions(builder); - listChangeStreams(builder); - listChangeStreamOptions(builder); + // Change stream is only supported in GOOGLE_STANDARD_SQL + if (dialect == Dialect.GOOGLE_STANDARD_SQL) { + listChangeStreams(builder); + listChangeStreamOptions(builder); + } Map> indexes = Maps.newHashMap(); listIndexes(indexes); listIndexColumns(builder, indexes); @@ -96,13 +107,29 @@ public Ddl scan() { } private void listDatabaseOptions(Ddl.Builder builder) { - ResultSet resultSet = - context.executeQuery( - Statement.newBuilder( - "SELECT t.option_name, t.option_type, t.option_value " - + " FROM information_schema.database_options AS t " - + " WHERE t.catalog_name = '' AND t.schema_name = ''") - .build()); + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = + Statement.of( + "SELECT t.option_name, t.option_type, t.option_value " + + " FROM information_schema.database_options AS t " + + " WHERE t.catalog_name = '' AND t.schema_name = ''"); + break; + case POSTGRESQL: + statement = + Statement.of( + "SELECT t.option_name, t.option_type, t.option_value " + + " FROM information_schema.database_options AS t " + + " WHERE t.schema_name NOT IN " + + "('information_schema', 'spanner_sys', 'pg_catalog')"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + + ResultSet resultSet = context.executeQuery(statement); ImmutableList.Builder options = ImmutableList.builder(); while (resultSet.next()) { @@ -123,26 +150,51 @@ private void listDatabaseOptions(Ddl.Builder builder) { } private void listTables(Ddl.Builder builder) { - Statement.Builder queryBuilder = - Statement.newBuilder( - "SELECT t.table_name, t.parent_table_name, t.on_delete_action FROM" - + " information_schema.tables AS t WHERE t.table_catalog = '' AND" - + " t.table_schema = ''"); - ResultSet resultSet = - context.executeQuery( + Statement.Builder queryBuilder; + + Statement preconditionStatement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + queryBuilder = + Statement.newBuilder( + "SELECT t.table_name, t.parent_table_name, t.on_delete_action FROM" + + " information_schema.tables AS t" + + " WHERE t.table_catalog = '' AND t.table_schema = ''"); + preconditionStatement = Statement.of( "SELECT COUNT(1) FROM INFORMATION_SCHEMA.COLUMNS c WHERE c.TABLE_CATALOG = '' AND" + " c.TABLE_SCHEMA = 'INFORMATION_SCHEMA' AND c.TABLE_NAME = 'TABLES' AND" - + " c.COLUMN_NAME = 'TABLE_TYPE';")); - // Returns a single row with a 1 if views are supported and a 0 if not. - resultSet.next(); - if (resultSet.getLong(0) == 0) { - LOG.info("INFORMATION_SCHEMA.TABLES.TABLE_TYPE is not present; assuming no views"); - } else { - queryBuilder.append(" AND t.table_type != 'VIEW'"); + + " c.COLUMN_NAME = 'TABLE_TYPE';"); + break; + case POSTGRESQL: + queryBuilder = + Statement.newBuilder( + "SELECT t.table_name, t.parent_table_name, t.on_delete_action FROM" + + " information_schema.tables AS t" + + " WHERE t.table_schema NOT IN " + + "('information_schema', 'spanner_sys', 'pg_catalog')"); + preconditionStatement = + Statement.of( + "SELECT COUNT(1) FROM INFORMATION_SCHEMA.COLUMNS c WHERE " + + " c.TABLE_SCHEMA = 'information_schema' AND c.TABLE_NAME = 'tables' AND" + + " c.COLUMN_NAME = 'table_type';"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); } - resultSet = context.executeQuery(queryBuilder.build()); + try (ResultSet resultSet = context.executeQuery(preconditionStatement)) { + // Returns a single row with a 1 if views are supported and a 0 if not. + resultSet.next(); + if (resultSet.getLong(0) == 0) { + LOG.info("INFORMATION_SCHEMA.TABLES.TABLE_TYPE is not present; assuming no views"); + } else { + queryBuilder.append(" AND t.table_type != 'VIEW'"); + } + } + + ResultSet resultSet = context.executeQuery(queryBuilder.build()); while (resultSet.next()) { String tableName = resultSet.getString(0); String parentTableName = resultSet.isNull(1) ? null : resultSet.getString(1); @@ -176,17 +228,37 @@ private void listTables(Ddl.Builder builder) { } private void listColumns(Ddl.Builder builder) { - ResultSet resultSet = - context.executeQuery( - Statement.newBuilder( - "SELECT c.table_name, c.column_name," - + " c.ordinal_position, c.spanner_type, c.is_nullable," - + " c.is_generated, c.generation_expression, c.is_stored" - + " FROM information_schema.columns as c" - + " WHERE c.table_catalog = '' AND c.table_schema = '' " - + " AND c.spanner_state = 'COMMITTED' " - + " ORDER BY c.table_name, c.ordinal_position") - .build()); + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = + Statement.of( + "SELECT c.table_name, c.column_name," + + " c.ordinal_position, c.spanner_type, c.is_nullable," + + " c.is_generated, c.generation_expression, c.is_stored" + + " FROM information_schema.columns as c" + + " WHERE c.table_catalog = '' AND c.table_schema = '' " + + " AND c.spanner_state = 'COMMITTED' " + + " ORDER BY c.table_name, c.ordinal_position"); + break; + case POSTGRESQL: + statement = + Statement.of( + "SELECT c.table_name, c.column_name," + + " c.ordinal_position, c.spanner_type, c.is_nullable," + + " c.is_generated, c.generation_expression, c.is_stored, c.column_default" + + " FROM information_schema.columns as c" + + " WHERE c.table_schema NOT IN " + + " ('information_schema', 'spanner_sys', 'pg_catalog') " + + " AND c.spanner_state = 'COMMITTED' " + + " ORDER BY c.table_name, c.ordinal_position"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + + ResultSet resultSet = context.executeQuery(statement); while (resultSet.next()) { String tableName = resultSet.getString(0); if (builder.hasView(tableName)) { @@ -201,6 +273,10 @@ private void listColumns(Ddl.Builder builder) { String generationExpression = resultSet.isNull(6) ? "" : resultSet.getString(6); boolean isStored = resultSet.isNull(7) ? false : resultSet.getString(7).equalsIgnoreCase("YES"); + String defaultExpression = + (dialect == Dialect.GOOGLE_STANDARD_SQL || resultSet.isNull(8)) + ? null + : resultSet.getString(8); builder .createTable(tableName) .column(columnName) @@ -209,21 +285,41 @@ private void listColumns(Ddl.Builder builder) { .isGenerated(isGenerated) .generationExpression(generationExpression) .isStored(isStored) + .defaultExpression(defaultExpression) .endColumn() .endTable(); } } private void listIndexes(Map> indexes) { - ResultSet resultSet = - context.executeQuery( + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = + Statement.of( + "SELECT t.table_name, t.index_name, t.parent_table_name, t.is_unique," + + " t.is_null_filtered" + + " FROM information_schema.indexes AS t" + + " WHERE t.table_catalog = '' AND t.table_schema = '' AND" + + " t.index_type='INDEX' AND t.spanner_is_managed = FALSE" + + " ORDER BY t.table_name, t.index_name"); + break; + case POSTGRESQL: + statement = Statement.of( - "SELECT t.table_name, t.index_name, t.parent_table_name," - + " t.is_unique, t.is_null_filtered" - + " FROM information_schema.indexes AS t " - + " WHERE t.table_catalog = '' AND t.table_schema = '' AND t.index_type='INDEX'" - + " AND t.spanner_is_managed = FALSE" - + " ORDER BY t.table_name, t.index_name")); + "SELECT t.table_name, t.index_name, t.parent_table_name, t.is_unique," + + " t.is_null_filtered, t.filter FROM information_schema.indexes AS t " + + " WHERE t.table_schema NOT IN " + + " ('information_schema', 'spanner_sys', 'pg_catalog')" + + " AND t.index_type='INDEX' AND t.spanner_is_managed = 'NO' " + + " ORDER BY t.table_name, t.index_name"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + + ResultSet resultSet = context.executeQuery(statement); while (resultSet.next()) { String tableName = resultSet.getString(0); String indexName = resultSet.getString(1); @@ -232,32 +328,60 @@ private void listIndexes(Map> indexe if (Strings.isNullOrEmpty(parent)) { parent = null; } - boolean unique = resultSet.getBoolean(3); - boolean nullFiltered = resultSet.getBoolean(4); + boolean unique = + (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? resultSet.getBoolean(3) + : resultSet.getString(3).equalsIgnoreCase("YES"); + boolean nullFiltered = + (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? resultSet.getBoolean(4) + : resultSet.getString(4).equalsIgnoreCase("YES"); + String filter = + (dialect == Dialect.GOOGLE_STANDARD_SQL || resultSet.isNull(5)) + ? null + : resultSet.getString(5); Map tableIndexes = indexes.computeIfAbsent(tableName, k -> Maps.newTreeMap()); tableIndexes.put( indexName, - Index.builder() + Index.builder(dialect) .name(indexName) .table(tableName) .unique(unique) .nullFiltered(nullFiltered) - .interleaveIn(parent)); + .interleaveIn(parent) + .filter(filter)); } } private void listIndexColumns( Ddl.Builder builder, Map> indexes) { - ResultSet resultSet = - context.executeQuery( + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = Statement.of( "SELECT t.table_name, t.column_name, t.column_ordering, t.index_name " + "FROM information_schema.index_columns AS t " - + "WHERE t.table_catalog = '' AND t.table_schema " - + "= '' ORDER BY t.table_name, t.index_name, t.ordinal_position")); + + "WHERE t.table_catalog = '' AND t.table_schema = '' " + + "ORDER BY t.table_name, t.index_name, t.ordinal_position"); + break; + case POSTGRESQL: + statement = + Statement.of( + "SELECT t.table_name, t.column_name, t.column_ordering, t.index_name " + + "FROM information_schema.index_columns AS t " + + "WHERE t.table_schema NOT IN " + + "('information_schema', 'spanner_sys', 'pg_catalog') " + + "ORDER BY t.table_name, t.index_name, t.ordinal_position"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + ResultSet resultSet = context.executeQuery(statement); while (resultSet.next()) { String tableName = resultSet.getString(0); String columnName = resultSet.getString(1); @@ -282,27 +406,57 @@ private void listIndexColumns( if (indexBuilder == null) { continue; } + IndexColumn.IndexColumnsBuilder indexColumnsBuilder = + indexBuilder.columns().create().name(columnName); if (ordering == null) { - indexBuilder.columns().storing(columnName).end(); - } else if (ordering.equalsIgnoreCase("ASC")) { - indexBuilder.columns().asc(columnName).end(); - } else if (ordering.equalsIgnoreCase("DESC")) { - indexBuilder.columns().desc(columnName).end(); + indexColumnsBuilder.storing(); + } else { + ordering = ordering.toUpperCase(); + if (ordering.startsWith("ASC")) { + indexColumnsBuilder.asc(); + } + if (ordering.startsWith("DESC")) { + indexColumnsBuilder.desc(); + } + if (ordering.endsWith("NULLS FIRST")) { + indexColumnsBuilder.nullsFirst(); + } + if (ordering.endsWith("NULLS LAST")) { + indexColumnsBuilder.nullsLast(); + } } + indexColumnsBuilder.endIndexColumn().end(); } } } private void listColumnOptions(Ddl.Builder builder) { - ResultSet resultSet = - context.executeQuery( - Statement.newBuilder( - "SELECT t.table_name, t.column_name," - + " t.option_name, t.option_type, t.option_value " - + " FROM information_schema.column_options AS t " - + " WHERE t.table_catalog = '' AND t.table_schema = ''" - + " ORDER BY t.table_name, t.column_name") - .build()); + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = + Statement.of( + "SELECT t.table_name, t.column_name, t.option_name, t.option_type," + + " t.option_value" + + " FROM information_schema.column_options AS t" + + " WHERE t.table_catalog = '' AND t.table_schema = ''" + + " ORDER BY t.table_name, t.column_name"); + break; + case POSTGRESQL: + statement = + Statement.of( + "SELECT t.table_name, t.column_name, t.option_name, t.option_type," + + " t.option_value" + + " FROM information_schema.column_options AS t" + + " WHERE t.table_schema NOT IN " + + " ('information_schema', 'spanner_sys', 'pg_catalog')" + + " ORDER BY t.table_name, t.column_name"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + ResultSet resultSet = context.executeQuery(statement); Map, ImmutableList.Builder> allOptions = Maps.newHashMap(); while (resultSet.next()) { @@ -341,8 +495,11 @@ private void listColumnOptions(Ddl.Builder builder) { } private void listForeignKeys(Map> foreignKeys) { - ResultSet resultSet = - context.executeQuery( + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = Statement.of( "SELECT rc.constraint_name," + " kcu1.table_name," @@ -365,7 +522,39 @@ private void listForeignKeys(Map ForeignKey.builder().name(name).table(table).referencedTable(referencedTable)); + k -> + ForeignKey.builder(dialect) + .name(name) + .table(table) + .referencedTable(referencedTable)); foreignKey.columnsBuilder().add(column); foreignKey.referencedColumnsBuilder().add(referencedColumn); } @@ -385,8 +578,12 @@ private void listForeignKeys(Map> listCheckConstraints() { Map> checkConstraints = Maps.newHashMap(); - ResultSet resultSet = - context.executeQuery( + + Statement statement; + + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = Statement.of( "SELECT ctu.TABLE_NAME," + " cc.CONSTRAINT_NAME," @@ -401,8 +598,31 @@ private Map> listCheckConstraints( + " AND ctu.table_schema = ''" + " AND ctu.constraint_catalog = ''" + " AND ctu.constraint_schema = ''" - + " AND cc.SPANNER_STATE = 'COMMITTED'" - + ";")); + + " AND cc.SPANNER_STATE = 'COMMITTED';"); + break; + case POSTGRESQL: + statement = + Statement.of( + "SELECT ctu.TABLE_NAME," + + " cc.CONSTRAINT_NAME," + + " cc.CHECK_CLAUSE" + + " FROM INFORMATION_SCHEMA.TABLE_CONSTRAINTS as ctu" + + " INNER JOIN INFORMATION_SCHEMA.CHECK_CONSTRAINTS as cc" + + " ON ctu.constraint_catalog = cc.constraint_catalog" + + " AND ctu.constraint_schema = cc.constraint_schema" + + " AND ctu.CONSTRAINT_NAME = cc.CONSTRAINT_NAME" + + " WHERE NOT STARTS_WITH(cc.CONSTRAINT_NAME, 'CK_IS_NOT_NULL_')" + + " AND ctu.table_catalog = ctu.constraint_catalog" + + " AND ctu.table_schema NOT IN" + + "('information_schema', 'spanner_sys', 'pg_catalog')" + + " AND ctu.table_schema = ctu.constraint_schema" + + " AND cc.SPANNER_STATE = 'COMMITTED';"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + + ResultSet resultSet = context.executeQuery(statement); while (resultSet.next()) { String table = resultSet.getString(0); String name = resultSet.getString(1); @@ -410,30 +630,56 @@ private Map> listCheckConstraints( Map tableCheckConstraints = checkConstraints.computeIfAbsent(table, k -> Maps.newTreeMap()); tableCheckConstraints.computeIfAbsent( - name, k -> CheckConstraint.builder().name(name).expression(expression).build()); + name, k -> CheckConstraint.builder(dialect).name(name).expression(expression).build()); } return checkConstraints; } private void listViews(Ddl.Builder builder) { - ResultSet resultSet = - context.executeQuery( - Statement.of( - "SELECT COUNT(1) FROM INFORMATION_SCHEMA.TABLES t WHERE t.TABLE_CATALOG = '' AND" - + " t.TABLE_SCHEMA = 'INFORMATION_SCHEMA' AND t.TABLE_NAME = 'VIEWS'")); - // Returns a single row with a 1 if views are supported and a 0 if not. - resultSet.next(); - if (resultSet.getLong(0) == 0) { - LOG.info("INFORMATION_SCHEMA.VIEWS is not present; assuming no views"); - return; - } + Statement queryStatement; + Statement preconditionStatement; - resultSet = - context.executeQuery( + switch (dialect) { + case GOOGLE_STANDARD_SQL: + queryStatement = + Statement.of( + "SELECT v.table_name, v.view_definition" + + " FROM information_schema.views AS v" + + " WHERE v.table_catalog = '' AND v.table_schema = ''"); + preconditionStatement = + Statement.of( + "SELECT COUNT(1)" + + " FROM INFORMATION_SCHEMA.TABLES t WHERE t.TABLE_CATALOG = '' AND" + + " t.TABLE_SCHEMA = 'INFORMATION_SCHEMA'" + + " AND t.TABLE_NAME = 'VIEWS'"); + break; + case POSTGRESQL: + queryStatement = Statement.of( "SELECT v.table_name, v.view_definition" + " FROM information_schema.views AS v" - + " WHERE v.table_catalog = '' AND v.table_schema = ''")); + + " WHERE v.table_schema NOT IN" + + " ('information_schema', 'spanner_sys', 'pg_catalog')"); + preconditionStatement = + Statement.of( + "SELECT COUNT(1)" + + " FROM INFORMATION_SCHEMA.TABLES t WHERE " + + " t.TABLE_SCHEMA = 'information_schema'" + + " AND t.TABLE_NAME = 'views'"); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + try (ResultSet resultSet = context.executeQuery(preconditionStatement)) { + // Returns a single row with a 1 if views are supported and a 0 if not. + resultSet.next(); + if (resultSet.getLong(0) == 0) { + LOG.info("INFORMATION_SCHEMA.VIEWS is not present; assuming no views"); + return; + } + } + + ResultSet resultSet = context.executeQuery(queryStatement); while (resultSet.next()) { String viewName = resultSet.getString(0); diff --git a/src/test/java/com/google/cloud/teleport/spanner/SpannerServerResource.java b/src/test/java/com/google/cloud/teleport/spanner/SpannerServerResource.java index ecc81de302..89c81e9dc8 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/SpannerServerResource.java +++ b/src/test/java/com/google/cloud/teleport/spanner/SpannerServerResource.java @@ -19,6 +19,7 @@ import com.google.cloud.spanner.DatabaseAdminClient; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerException; import com.google.cloud.spanner.SpannerOptions; @@ -26,6 +27,7 @@ import com.google.cloud.spanner.TransactionRunner; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.RandomInsertMutationGenerator; +import java.util.Arrays; import java.util.Iterator; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.spanner.MutationGroup; @@ -55,7 +57,11 @@ public SpannerServerResource() { protected void before() { SpannerOptions spannerOptions; if (EMULATOR_HOST == null) { - spannerOptions = SpannerOptions.newBuilder().setProjectId(projectId).setHost(host).build(); + spannerOptions = + SpannerOptions.newBuilder() + .setProjectId(projectId) + .setHost(host) + .build(); } else { spannerOptions = SpannerOptions.newBuilder() @@ -77,6 +83,20 @@ public void createDatabase(String dbName, Iterable ddlStatements) throws databaseAdminClient.createDatabase(instanceId, dbName, ddlStatements).get(); } + public void createPgDatabase(String dbName, Iterable ddlStatements) throws Exception { + databaseAdminClient + .createDatabase( + databaseAdminClient + .newDatabaseBuilder(DatabaseId.of(projectId, instanceId, dbName)) + .setDialect(Dialect.POSTGRESQL) + .build(), + Arrays.asList()) + .get(); + if (ddlStatements.iterator().hasNext()) { + databaseAdminClient.updateDatabaseDdl(instanceId, dbName, ddlStatements, null).get(); + } + } + public void updateDatabase(String dbName, Iterable ddlStatements) throws Exception { databaseAdminClient.updateDatabaseDdl(instanceId, dbName, ddlStatements, null).get(); } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java index 6aca2fb98a..e6fb22ea9c 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java @@ -71,6 +71,14 @@ private Ddl getDatabaseDdl() { return scanner.scan(); } + private Ddl getPgDatabaseDdl() { + BatchClient batchClient = spannerServer.getBatchClient(dbId); + BatchReadOnlyTransaction batchTx = + batchClient.batchReadOnlyTransaction(TimestampBound.strong()); + InformationSchemaScanner scanner = new InformationSchemaScanner(batchTx, Dialect.POSTGRESQL); + return scanner.scan(); + } + @Test public void emptyDatabase() throws Exception { spannerServer.createDatabase(dbId, Collections.emptyList()); @@ -78,6 +86,13 @@ public void emptyDatabase() throws Exception { assertThat(ddl, equalTo(Ddl.builder().build())); } + @Test + public void pgEmptyDatabase() throws Exception { + spannerServer.createPgDatabase(dbId, Collections.emptyList()); + Ddl ddl = getPgDatabaseDdl(); + assertThat(ddl, equalTo(Ddl.builder(Dialect.POSTGRESQL).build())); + } + @Test public void tableWithAllTypes() throws Exception { String allTypes = @@ -156,6 +171,85 @@ public void tableWithAllTypes() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(allTypes)); } + @Test + public void tableWithAllPgTypes() throws Exception { + String allTypes = + "CREATE TABLE \"alltypes\" (" + + " \"first_name\" character varying NOT NULL," + + " \"last_name\" character varying(5) NOT NULL," + + " \"id\" bigint NOT NULL," + + " \"bool_field\" boolean," + + " \"int64_field\" bigint," + + " \"float64_field\" double precision," + + " \"string_field\" character varying(76)," + + " \"bytes_field\" bytea," + + " \"numeric_field\" numeric," + + " \"timestamp_field\" timestamp with time zone," + + " \"date_field\" date," + + " \"arr_bool_field\" boolean[]," + + " \"arr_int64_field\" bigint[]," + + " \"arr_float64_field\" double precision[]," + + " \"arr_string_field\" character varying(15)[]," + + " \"arr_bytes_field\" bytea[]," + + " \"arr_timestamp_field\" timestamp with time zone[]," + + " \"arr_date_field\" date[]," + + " \"arr_numeric_field\" numeric[]," + + " PRIMARY KEY (\"first_name\", \"last_name\", \"id\")" + + " )"; + + spannerServer.createPgDatabase(dbId, Collections.singleton(allTypes)); + Ddl ddl = getPgDatabaseDdl(); + + assertThat(ddl.allTables(), hasSize(1)); + assertThat(ddl.table("alltypes"), notNullValue()); + assertThat(ddl.table("aLlTYPeS"), notNullValue()); + + Table table = ddl.table("alltypes"); + assertThat(table.columns(), hasSize(19)); + + // Check case sensitiveness. + assertThat(table.column("first_name"), notNullValue()); + assertThat(table.column("fIrst_NaME"), notNullValue()); + assertThat(table.column("last_name"), notNullValue()); + assertThat(table.column("LAST_name"), notNullValue()); + + // Check types/sizes. + assertThat(table.column("bool_field").type(), equalTo(Type.pgBool())); + assertThat(table.column("int64_field").type(), equalTo(Type.pgInt8())); + assertThat(table.column("float64_field").type(), equalTo(Type.pgFloat8())); + assertThat(table.column("string_field").type(), equalTo(Type.pgVarchar())); + assertThat(table.column("string_field").size(), equalTo(76)); + assertThat(table.column("bytes_field").type(), equalTo(Type.pgBytea())); + assertThat(table.column("timestamp_field").type(), equalTo(Type.pgTimestamptz())); + assertThat(table.column("numeric_field").type(), equalTo(Type.pgNumeric())); + assertThat(table.column("date_field").type(), equalTo(Type.pgDate())); + assertThat(table.column("arr_bool_field").type(), equalTo(Type.pgArray(Type.pgBool()))); + assertThat(table.column("arr_int64_field").type(), equalTo(Type.pgArray(Type.pgInt8()))); + assertThat(table.column("arr_float64_field").type(), equalTo(Type.pgArray(Type.pgFloat8()))); + assertThat(table.column("arr_string_field").type(), equalTo(Type.pgArray(Type.pgVarchar()))); + assertThat(table.column("arr_string_field").size(), equalTo(15)); + assertThat(table.column("arr_bytes_field").type(), equalTo(Type.pgArray(Type.pgBytea()))); + assertThat( + table.column("arr_timestamp_field").type(), equalTo(Type.pgArray(Type.pgTimestamptz()))); + assertThat(table.column("arr_date_field").type(), equalTo(Type.pgArray(Type.pgDate()))); + assertThat(table.column("arr_numeric_field").type(), equalTo(Type.pgArray(Type.pgNumeric()))); + + // Check not-null. Primary keys are implictly forced to be not-null. + assertThat(table.column("first_name").notNull(), is(true)); + assertThat(table.column("last_name").notNull(), is(true)); + assertThat(table.column("id").notNull(), is(true)); + + // Check primary key. + assertThat(table.primaryKeys(), hasSize(3)); + List pk = table.primaryKeys(); + assertThat(pk.get(0).name(), equalTo("first_name")); + assertThat(pk.get(1).name(), equalTo("last_name")); + assertThat(pk.get(2).name(), equalTo("id")); + + // Verify pretty print. + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(allTypes)); + } + @Test public void simpleView() throws Exception { String tableDef = @@ -229,6 +323,58 @@ public void interleavedIn() throws Exception { assertThat(ddl.table("level2_1").onDeleteCascade(), is(true)); } + @Test + public void pgInterleavedIn() throws Exception { + List statements = + Arrays.asList( + " CREATE TABLE level0 (" + + " id0 bigint NOT NULL," + + " val0 character varying," + + " PRIMARY KEY (id0)" + + " )", + " CREATE TABLE level1 (" + + " id0 bigint NOT NULL," + + " id1 bigint NOT NULL," + + " val1 character varying," + + " PRIMARY KEY (id0, id1)" + + " ) INTERLEAVE IN PARENT level0", + " CREATE TABLE level2 (" + + " id0 bigint NOT NULL," + + " id1 bigint NOT NULL," + + " id2 bigint NOT NULL," + + " val2 character varying," + + " PRIMARY KEY (id0, id1, id2)" + + " ) INTERLEAVE IN PARENT level1", + " CREATE TABLE level2_1 (" + + " id0 bigint NOT NULL," + + " id1 bigint NOT NULL," + + " id2_1 bigint NOT NULL," + + " val2 character varying," + + " PRIMARY KEY (id0, id1, id2_1)" + + " ) INTERLEAVE IN PARENT level1 ON DELETE CASCADE"); + + spannerServer.createPgDatabase(dbId, statements); + Ddl ddl = getPgDatabaseDdl(); + + assertThat(ddl.allTables(), hasSize(4)); + HashMultimap levels = ddl.perLevelView(); + assertThat(levels.get(0), hasSize(1)); + assertThat(levels.get(1), hasSize(1)); + assertThat(levels.get(2), hasSize(2)); + assertThat(levels.get(3), hasSize(0)); + assertThat(levels.get(4), hasSize(0)); + assertThat(levels.get(5), hasSize(0)); + assertThat(levels.get(6), hasSize(0)); + assertThat(levels.get(7), hasSize(0)); + + assertThat(ddl.table("lEVEl0").interleaveInParent(), nullValue()); + assertThat(ddl.table("level1").interleaveInParent(), equalTo("level0")); + assertThat(ddl.table("level2").interleaveInParent(), equalTo("level1")); + assertThat(ddl.table("level2").onDeleteCascade(), is(false)); + assertThat(ddl.table("level2_1").interleaveInParent(), equalTo("level1")); + assertThat(ddl.table("level2_1").onDeleteCascade(), is(true)); + } + @Test public void reserved() throws Exception { String statement = @@ -252,6 +398,30 @@ public void reserved() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(statement)); } + @Test + public void pgReserved() throws Exception { + String statement = + "CREATE TABLE \"where\" (" + + " \"JOIN\" character varying NOT NULL," + + " \"TABLE\" bigint," + + " \"NULL\" bigint NOT NULL," + + " PRIMARY KEY (\"NULL\")" + + " )"; + + spannerServer.createPgDatabase(dbId, Collections.singleton(statement)); + Ddl ddl = getPgDatabaseDdl(); + + assertThat(ddl.allTables(), hasSize(1)); + + assertThat(ddl.table("where"), notNullValue()); + Table table = ddl.table("where"); + assertThat(table.column("JOIN"), notNullValue()); + assertThat(table.column("Table"), notNullValue()); + assertThat(table.column("NULL"), notNullValue()); + + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(statement)); + } + @Test public void indexes() throws Exception { // Prefix indexes to ensure ordering. @@ -273,6 +443,31 @@ public void indexes() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); } + @Test + public void pgIndexes() throws Exception { + // Prefix indexes to ensure ordering. + // Unique index is implicitly null-filtered. + List statements = + Arrays.asList( + "CREATE TABLE \"Users\" (" + + " \"id\" bigint NOT NULL," + + " \"first_name\" character varying(10)," + + " \"last_name\" character varying," + + " \"AGE\" bigint," + + " PRIMARY KEY (\"id\")" + + " )", + " CREATE UNIQUE INDEX \"a_last_name_idx\" ON \"Users\"(\"last_name\" ASC) INCLUDE" + + " (\"first_name\") WHERE first_name IS NOT NULL AND last_name IS NOT" + + " NULL", + " CREATE INDEX \"b_age_idx\" ON \"Users\"(\"AGE\" DESC) WHERE \"AGE\" IS NOT NULL", + " CREATE UNIQUE INDEX \"c_first_name_idx\" ON \"Users\"(\"first_name\" ASC) WHERE" + + " first_name IS NOT NULL"); + + spannerServer.createPgDatabase(dbId, statements); + Ddl ddl = getPgDatabaseDdl(); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); + } + @Test public void foreignKeys() throws Exception { List statements = @@ -294,6 +489,29 @@ public void foreignKeys() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); } + @Test + public void pgForeignKeys() throws Exception { + List statements = + Arrays.asList( + "CREATE TABLE \"Ref\" (" + + " \"id1\" bigint NOT NULL," + + " \"id2\" bigint NOT NULL," + + " PRIMARY KEY (\"id1\", \"id2\")" + + " )", + " CREATE TABLE \"Tab\" (" + + " \"key\" bigint NOT NULL," + + " \"id1\" bigint NOT NULL," + + " \"id2\" bigint NOT NULL," + + " PRIMARY KEY (\"key\")" + + " )", + " ALTER TABLE \"Tab\" ADD CONSTRAINT \"fk\" FOREIGN KEY (\"id1\", \"id2\")" + + " REFERENCES \"Ref\" (\"id2\", \"id1\")"); + + spannerServer.createPgDatabase(dbId, statements); + Ddl ddl = getPgDatabaseDdl(); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); + } + // TODO: enable this test once CHECK constraints are enabled // @Test public void checkConstraints() throws Exception { @@ -310,6 +528,22 @@ public void checkConstraints() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); } + @Test + public void pgCheckConstraints() throws Exception { + List statements = + Arrays.asList( + "CREATE TABLE \"T\" (" + + " \"id\" bigint NOT NULL," + + " \"A\" bigint NOT NULL," + + " CONSTRAINT \"ck\" CHECK ((\"A\" > '0'::bigint))," + + " PRIMARY KEY (\"id\")" + + " )"); + + spannerServer.createPgDatabase(dbId, statements); + Ddl ddl = getPgDatabaseDdl(); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); + } + @Test public void commitTimestamp() throws Exception { String statement = @@ -338,6 +572,31 @@ public void generatedColumns() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(statement)); } + @Test + public void pgGeneratedColumns() throws Exception { + String statement = + "CREATE TABLE \"T\" ( \"id\" bigint NOT NULL," + + " \"generated\" bigint NOT NULL GENERATED ALWAYS AS" + + " (\"id\"/1) STORED," + + " PRIMARY KEY (\"id\"))"; + + spannerServer.createPgDatabase(dbId, Collections.singleton(statement)); + Ddl ddl = getPgDatabaseDdl(); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(statement)); + } + + @Test + public void pgDefaultColumns() throws Exception { + String statement = + "CREATE TABLE \"T\" ( \"id\" bigint NOT NULL," + + " \"generated\" bigint NOT NULL DEFAULT '10'::bigint," + + " PRIMARY KEY (\"id\") )"; + + spannerServer.createPgDatabase(dbId, Collections.singleton(statement)); + Ddl ddl = getPgDatabaseDdl(); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(statement)); + } + @Test public void databaseOptions() throws Exception { List statements = @@ -361,6 +620,30 @@ public void databaseOptions() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); } + @Test + public void pgDatabaseOptions() throws Exception { + List statements = + Arrays.asList( + "ALTER DATABASE \"" + dbId + "\" SET spanner.version_retention_period = '5d'\n", + "CREATE TABLE \"Users\" (" + + " \"id\" bigint NOT NULL," + + " \"first_name\" character varying(10)," + + " \"last_name\" character varying," + + " \"age\" bigint," + + " PRIMARY KEY (\"id\")" + + " ) ", + " CREATE INDEX \"a_last_name_idx\" ON " + + " \"Users\"(\"last_name\" ASC) INCLUDE (\"first_name\")", + " CREATE INDEX \"b_age_idx\" ON \"Users\"(\"age\" DESC)", + " CREATE INDEX \"c_first_name_idx\" ON \"Users\"(\"first_name\" ASC)"); + + spannerServer.createPgDatabase(dbId, statements); + Ddl ddl = getPgDatabaseDdl(); + String alterStatement = statements.get(0); + statements.set(0, alterStatement.replace(dbId, "%db_name%")); + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(String.join("", statements))); + } + @Test public void changeStreams() throws Exception { List statements = diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java index 53d300cc17..4e351dd899 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java @@ -22,16 +22,19 @@ import com.google.cloud.Timestamp; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Value; +import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.function.Supplier; import java.util.stream.Stream; import org.apache.beam.sdk.io.gcp.spanner.MutationGroup; @@ -143,10 +146,20 @@ private static class TableSupplier implements Supplier { public TableSupplier(Table table) { this.table = checkNotNull(table); RandomValueGenerator randomValueGenerator = RandomValueGenerator.defaultInstance(); - + Dialect dialect = table.dialect(); + Set primaryKeyNameSet = new HashSet<>(); + if (dialect == Dialect.POSTGRESQL) { + for (IndexColumn primaryKey : table.primaryKeys()) { + primaryKeyNameSet.add(primaryKey.name()); + } + } for (Column column : table.columns()) { if (!column.isGenerated()) { - valueGenerators.put(column.name(), randomValueGenerator.valueStream(column).iterator()); + valueGenerators.put( + column.name(), + randomValueGenerator + .valueStream(column, primaryKeyNameSet.contains(column.name())) + .iterator()); } } } @@ -181,6 +194,7 @@ public Mutation generateMutation(Map overrides) { ByteArray bytes = value.isNull() ? null : value.getBytes(); builder.set(columnName).to(bytes); break; + case PG_NUMERIC: case STRING: String string = value.isNull() ? null : value.getString(); builder.set(columnName).to(string); @@ -193,6 +207,10 @@ public Mutation generateMutation(Map overrides) { Date date = value.isNull() ? null : value.getDate(); builder.set(columnName).to(date); break; + case NUMERIC: + BigDecimal numeric = value.isNull() ? null : value.getNumeric(); + builder.set(columnName).to(numeric); + break; case ARRAY: switch (value.getType().getArrayElementType().getCode()) { case BOOL: @@ -223,6 +241,10 @@ public Mutation generateMutation(Map overrides) { List dates = value.isNull() ? null : value.getDateArray(); builder.set(columnName).toDateArray(dates); break; + case NUMERIC: + List numerics = value.isNull() ? null : value.getNumericArray(); + builder.set(columnName).toNumericArray(numerics); + break; } break; default: diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java index 878dacd0f4..e294713747 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java @@ -19,6 +19,7 @@ import com.google.cloud.Date; import com.google.cloud.Timestamp; import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; import java.util.ArrayList; import java.util.List; @@ -43,11 +44,11 @@ public RandomValueGenerator(Random random, int nullThreshold, int arrayNullThres this.arrayNullThreshold = arrayNullThreshold; } - public Stream valueStream(Column column) { + public Stream valueStream(Column column, boolean notNull) { return Stream.generate( () -> { int threshold = nullThreshold; - if (column.notNull()) { + if (notNull || column.notNull()) { threshold = -1; } if (random.nextInt(100) < threshold) { @@ -60,35 +61,60 @@ public Stream valueStream(Column column) { private Value generateNullValue(Type type) { switch (type.getCode()) { case BOOL: + case PG_BOOL: return Value.bool(null); case INT64: + case PG_INT8: return Value.int64(null); case FLOAT64: + case PG_FLOAT8: return Value.float64(null); case BYTES: + case PG_BYTEA: return Value.bytes(null); case STRING: + case PG_TEXT: + case PG_VARCHAR: return Value.string(null); case DATE: + case PG_DATE: return Value.date(null); case TIMESTAMP: + case PG_TIMESTAMPTZ: return Value.timestamp(null); + case NUMERIC: + return Value.numeric(null); + case PG_NUMERIC: + return Value.pgNumeric(null); case ARRAY: + case PG_ARRAY: switch (type.getArrayElementType().getCode()) { case BOOL: + case PG_BOOL: return Value.boolArray((boolean[]) null); case INT64: + case PG_INT8: return Value.int64Array((long[]) null); case FLOAT64: + case PG_FLOAT8: return Value.float64Array((double[]) null); case BYTES: + case PG_BYTEA: return Value.bytesArray(null); case STRING: + case PG_TEXT: + case PG_VARCHAR: return Value.stringArray(null); case DATE: + case PG_DATE: return Value.dateArray(null); case TIMESTAMP: + case PG_TIMESTAMPTZ: return Value.timestampArray(null); + case NUMERIC: + return Value.numericArray(null); + case PG_NUMERIC: + return Value.pgNumericArray(null); } } throw new IllegalArgumentException("Unexpected type " + type); @@ -97,25 +123,35 @@ private Value generateNullValue(Type type) { private Value generate(Column column) { Type type = column.type(); - if (type.getCode() != Type.Code.ARRAY) { + if (type.getCode() != Type.Code.ARRAY && type.getCode() != Type.Code.PG_ARRAY) { return generateScalar(column); } switch (type.getArrayElementType().getCode()) { case BOOL: + case PG_BOOL: return Value.boolArray(generateList(random::nextBoolean)); case INT64: + case PG_INT8: return Value.int64Array(generateList(random::nextLong)); case FLOAT64: + case PG_FLOAT8: return Value.float64Array(generateList(random::nextDouble)); case BYTES: + case PG_BYTEA: return Value.bytesArray(generateList(() -> randomByteArray(column.size()))); case STRING: + case PG_VARCHAR: + case PG_TEXT: return Value.stringArray(generateList(() -> randomString(column.size()))); case DATE: + case PG_DATE: return Value.dateArray(generateList(this::randomDate)); case TIMESTAMP: + case PG_TIMESTAMPTZ: return Value.timestampArray(generateList(this::randomTimestamp)); + case PG_NUMERIC: + return Value.pgNumericArray(generateList(this::randomPgNumeric)); } throw new IllegalArgumentException("Unexpected type " + type); } @@ -134,31 +170,62 @@ private Value generateScalar(Column column) { Type type = column.type(); switch (type.getCode()) { case BOOL: + case PG_BOOL: return Value.bool(random.nextBoolean()); case INT64: + case PG_INT8: return Value.int64(random.nextLong()); case FLOAT64: + case PG_FLOAT8: return Value.float64(random.nextDouble()); case BYTES: + case PG_BYTEA: { return Value.bytes(randomByteArray(column.size())); } case STRING: + case PG_VARCHAR: + case PG_TEXT: { return Value.string(randomString(column.size())); } case DATE: + case PG_DATE: { return Value.date(randomDate()); } case TIMESTAMP: + case PG_TIMESTAMPTZ: { return Value.timestamp(randomTimestamp()); } + case PG_NUMERIC: + return Value.pgNumeric(randomPgNumeric()); } throw new IllegalArgumentException("Unexpected type " + type); } + private String randomPgNumeric() { + int leftSize = random.nextInt(NumericUtils.PG_MAX_PRECISION - NumericUtils.PG_MAX_SCALE) + 1; + int rightSize = random.nextInt(NumericUtils.PG_MAX_SCALE + 1); + StringBuilder sb = new StringBuilder(); + if (leftSize == 1) { + sb.append(0); + } else { + sb.append(random.nextInt(9) + 1); + } + for (int i = 1; i < leftSize; i++) { + sb.append(random.nextInt(10)); + } + if (rightSize > 0) { + sb.append("."); + for (int i = 0; i < rightSize; i++) { + sb.append(random.nextInt(10)); + } + } + return sb.toString(); + } + private ByteArray randomByteArray(Integer size) { size = size == -1 ? 20 : size; byte[] bytes = new byte[size]; From 5e88b9b1bac8079394411ae8319deac66c40faf6 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 15:54:34 -0700 Subject: [PATCH 131/145] Support change streams in Cloud Spanner Import/Export pipelines. - The export pipeline will export only the schema definitions of the change streams to the Avro files and generate the corresponding JSON manifest files. It will not export any change streams data. - The import pipeline will extract the schema definitions of the change streams from the Avro files and create the change streams in the destination database at the end of the pipeline, after finishing the data load. PiperOrigin-RevId: 447848070 --- .../spanner/AvroSchemaToDdlConverter.java | 4 +- .../cloud/teleport/spanner/AvroUtil.java | 3 + .../spanner/DdlToAvroSchemaConverter.java | 2 +- .../teleport/spanner/ExportTransform.java | 65 ++++++++-- .../teleport/spanner/ImportPipeline.java | 9 ++ .../teleport/spanner/ImportTransform.java | 59 ++++++++- src/main/proto/export.proto | 12 +- .../cloud/teleport/spanner/CopyDbTest.java | 40 +++++++ .../spanner/ExportRelatedTablesCheckTest.java | 1 + .../teleport/spanner/ExportTimestampTest.java | 1 + .../teleport/spanner/ExportTransformTest.java | 15 ++- .../teleport/spanner/ImportFromAvroTest.java | 113 ++++++++++++++++++ 12 files changed, 302 insertions(+), 22 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java index 4951f9527b..64fffeafc6 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java @@ -42,7 +42,7 @@ public Ddl toDdl(Collection avroSchemas) { for (Schema schema : avroSchemas) { if (schema.getProp("spannerViewQuery") != null) { builder.addView(toView(null, schema)); - } else if (schema.getProp("spannerChangeStreamForClause") != null) { + } else if (schema.getProp(AvroUtil.CHANGE_STREAM_FOR_CLAUSE) != null) { builder.addChangeStream(toChangeStream(null, schema)); } else { builder.addTable(toTable(null, schema)); @@ -73,7 +73,7 @@ public ChangeStream toChangeStream(String changeStreamName, Schema schema) { ChangeStream.Builder builder = ChangeStream.builder() .name(changeStreamName) - .forClause(schema.getProp("spannerChangeStreamForClause")); + .forClause(schema.getProp(AvroUtil.CHANGE_STREAM_FOR_CLAUSE)); ImmutableList.Builder changeStreamOptions = ImmutableList.builder(); for (int i = 0; ; i++) { diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java b/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java index 714ce7d8ea..766b0bf3ed 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java @@ -22,6 +22,9 @@ public class AvroUtil { private AvroUtil() {} + // The property name for change stream for-clause in Avro schema. + public static final String CHANGE_STREAM_FOR_CLAUSE = "spannerChangeStreamForClause"; + public static Schema unpackNullable(Schema schema) { if (schema.getType() != Schema.Type.UNION) { return null; diff --git a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java index 52d171462e..ec2da1eddb 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java @@ -118,7 +118,7 @@ public Collection convert(Ddl ddl) { recordBuilder.prop("googleFormatVersion", version); recordBuilder.prop("googleStorage", "CloudSpanner"); recordBuilder.prop( - "spannerChangeStreamForClause", + AvroUtil.CHANGE_STREAM_FOR_CLAUSE, changeStream.forClause() == null ? "" : changeStream.forClause()); if (changeStream.options() != null) { for (int i = 0; i < changeStream.options().size(); i++) { diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java index 9e9218dcfd..be55b9e35c 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java @@ -22,6 +22,7 @@ import com.google.cloud.spanner.TimestampBound; import com.google.cloud.teleport.spanner.ExportProtos.Export; import com.google.cloud.teleport.spanner.ExportProtos.TableManifest; +import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.templates.common.SpannerConverters.CreateTransactionFnWithTimestamp; @@ -290,6 +291,21 @@ public void processElement(ProcessContext c) { } })); + PCollection allChangeStreamNames = + ddl.apply( + "List all change stream names", + ParDo.of( + new DoFn() { + + @ProcessElement + public void processElement(ProcessContext c) { + Ddl ddl = c.element(); + for (ChangeStream changeStream : ddl.changeStreams()) { + c.output(changeStream.name()); + } + } + })); + // Generate a unique output directory name. final PCollectionView outputDirectoryName = p.apply(Create.of(1)) @@ -400,8 +416,33 @@ public void processElement(ProcessContext c) { } })); - emptyTablesAndViews = - emptyTablesAndViews.apply( + PCollection>> changeStreams = + allChangeStreamNames.apply( + "Export change streams", + ParDo.of( + new DoFn>>() { + + @ProcessElement + public void processElement(ProcessContext c) { + String changeStreamName = c.element(); + LOG.info("Exporting change stream: " + changeStreamName); + // This file will contain the schema definition for the change stream. + c.output( + KV.of( + changeStreamName, + Collections.singleton(changeStreamName + ".avro-00000-of-00001"))); + } + })); + + // Empty tables, views and change streams are handled together, because we export them as empty + // Avro files that only contain the Avro schemas. + PCollection>> emptySchemaFiles = + PCollectionList.of(emptyTablesAndViews) + .and(changeStreams) + .apply("Combine all empty schema files", Flatten.pCollections()); + + emptySchemaFiles = + emptySchemaFiles.apply( "Save empty schema files", ParDo.of( new DoFn>, KV>>() { @@ -411,10 +452,11 @@ public void processElement(ProcessContext c) { Map schemaMap = c.sideInput(avroSchemas); KV> kv = c.element(); - String tableName = kv.getKey(); + String objectName = kv.getKey(); String fileName = kv.getValue().iterator().next(); - Schema schema = schemaMap.get(tableName).get(); + Schema schema = schemaMap.get(objectName).get(); + DatumWriter datumWriter = new GenericDatumWriter<>(schema); Path fullPath = @@ -427,7 +469,7 @@ public void processElement(ProcessContext c) { } catch (IOException e) { throw new RuntimeException(e); } - c.output(KV.of(tableName, Collections.singleton(fullPath.toString()))); + c.output(KV.of(objectName, Collections.singleton(fullPath.toString()))); } /** @@ -473,6 +515,7 @@ private OutputStream createOutputStream(Path outputPath, ProcessContext c) return Channels.newOutputStream(gcsChannel); } else { // Avro file is created on local filesystem (for testing). + Files.createDirectories(outputPath.getParent()); return Files.newOutputStream(outputPath); } } @@ -481,7 +524,7 @@ private OutputStream createOutputStream(Path outputPath, ProcessContext c) PCollection>> allFiles = PCollectionList.of(tableFiles) - .and(emptyTablesAndViews) + .and(emptySchemaFiles) .apply("Combine all files", Flatten.pCollections()); PCollection> tableManifests = @@ -704,10 +747,16 @@ public CreateDatabaseManifest(PCollectionView ddlView) { @ProcessElement public void processElement( - @Element List metadataTables, OutputReceiver out, ProcessContext c) { + @Element List exportMetadata, OutputReceiver out, ProcessContext c) { Ddl ddl = c.sideInput(ddlView); ExportProtos.Export.Builder exportManifest = ExportProtos.Export.newBuilder(); - exportManifest.addAllTables(metadataTables); + for (Export.Table obj : exportMetadata) { + if (ddl.changeStream(obj.getName()) != null) { + exportManifest.addChangeStreams(obj); + } else { + exportManifest.addTables(obj); + } + } exportManifest.addAllDatabaseOptions(ddl.databaseOptions()); try { out.output(JsonFormat.printer().print(exportManifest.build())); diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java index f38fa17e4a..3e29e26fe5 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportPipeline.java @@ -76,6 +76,14 @@ public interface Options extends PipelineOptions { void setWaitForForeignKeys(ValueProvider value); + @Description( + "By default the import pipeline is blocked on change stream creation. If false, it may" + + " complete with change streams still being created in the background.") + @Default.Boolean(true) + ValueProvider getWaitForChangeStreams(); + + void setWaitForChangeStreams(ValueProvider value); + @Description( "Indexes and Foreign keys are created after dataload. If there are more than " + "40 DDL statements to be executed after dataload, it is preferable to create the " @@ -136,6 +144,7 @@ public static void main(String[] args) { options.getInputDir(), options.getWaitForIndexes(), options.getWaitForForeignKeys(), + options.getWaitForChangeStreams(), options.getEarlyIndexCreateFlag(), options.getDDLCreationTimeoutInMinutes())); diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java index d28383bbc0..d532efd993 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java @@ -20,6 +20,7 @@ import com.google.cloud.spanner.Mutation; import com.google.cloud.teleport.spanner.ExportProtos.Export; import com.google.cloud.teleport.spanner.ExportProtos.TableManifest; +import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.common.annotations.VisibleForTesting; @@ -100,6 +101,7 @@ public class ImportTransform extends PTransform { // are finished, respectively. private final ValueProvider waitForIndexes; private final ValueProvider waitForForeignKeys; + private final ValueProvider waitForChangeStreams; private final ValueProvider earlyIndexCreateFlag; private final ValueProvider ddlCreationTimeoutInMinutes; @@ -108,12 +110,14 @@ public ImportTransform( ValueProvider importDirectory, ValueProvider waitForIndexes, ValueProvider waitForForeignKeys, + ValueProvider waitForChangeStreams, ValueProvider earlyIndexCreateFlag, ValueProvider ddlCreationTimeoutInMinutes) { this.spannerConfig = spannerConfig; this.importDirectory = importDirectory; this.waitForIndexes = waitForIndexes; this.waitForForeignKeys = waitForForeignKeys; + this.waitForChangeStreams = waitForChangeStreams; this.earlyIndexCreateFlag = earlyIndexCreateFlag; this.ddlCreationTimeoutInMinutes = ddlCreationTimeoutInMinutes; } @@ -132,7 +136,7 @@ public PDone expand(PBegin begin) { PCollection> schemas = tableFiles .apply( - "File per table", + "File per table, view or change stream", ParDo.of( new DoFn>, KV>() { @@ -179,6 +183,10 @@ public void processElement(ProcessContext c) { createTableOutput .get(CreateTables.getPendingForeignKeysTag()) .apply("As Foreign keys view", View.asSingleton()); + final PCollectionView> pendingChangeStreams = + createTableOutput + .get(CreateTables.getPendingChangeStreamsTag()) + .apply("As change streams view", View.asSingleton()); PCollectionView ddlView = ddl.apply("Cloud Spanner DDL as view", View.asSingleton()); @@ -259,7 +267,10 @@ public void processElement(ProcessContext c) { "Create Indexes", new ApplyDDLTransform(spannerConfig, pendingIndexes, waitForIndexes)) .apply( "Add Foreign Keys", - new ApplyDDLTransform(spannerConfig, pendingForeignKeys, waitForForeignKeys)); + new ApplyDDLTransform(spannerConfig, pendingForeignKeys, waitForForeignKeys)) + .apply( + "Create Change Streams", + new ApplyDDLTransform(spannerConfig, pendingChangeStreams, waitForChangeStreams)); return PDone.in(begin.getPipeline()); } @@ -367,10 +378,16 @@ public static TupleTag> getPendingForeignKeysTag() { return pendingForeignKeysTag; } + public static TupleTag> getPendingChangeStreamsTag() { + return pendingChangeStreamsTag; + } + private static final TupleTag ddlObjectTag = new TupleTag() {}; private static final TupleTag> pendingIndexesTag = new TupleTag>() {}; private static final TupleTag> pendingForeignKeysTag = new TupleTag>() {}; + private static final TupleTag> pendingChangeStreamsTag = + new TupleTag>() {}; public CreateTables( SpannerConfig spannerConfig, @@ -417,11 +434,15 @@ public void processElement(ProcessContext c) { Schema.Parser parser = new Schema.Parser(); List> missingTables = new ArrayList<>(); List> missingViews = new ArrayList<>(); + List> missingChangeStreams = new ArrayList<>(); for (KV kv : avroSchemas) { if (informationSchemaDdl.table(kv.getKey()) == null - && informationSchemaDdl.view(kv.getKey()) == null) { + && informationSchemaDdl.view(kv.getKey()) == null + && informationSchemaDdl.changeStream(kv.getKey()) == null) { Schema schema = parser.parse(kv.getValue()); - if (schema.getProp("spannerViewQuery") != null) { + if (schema.getProp(AvroUtil.CHANGE_STREAM_FOR_CLAUSE) != null) { + missingChangeStreams.add(KV.of(kv.getKey(), schema)); + } else if (schema.getProp("spannerViewQuery") != null) { missingViews.add(KV.of(kv.getKey(), schema)); } else { missingTables.add(KV.of(kv.getKey(), schema)); @@ -431,6 +452,7 @@ public void processElement(ProcessContext c) { AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(); List createIndexStatements = new ArrayList<>(); List createForeignKeyStatements = new ArrayList<>(); + List createChangeStreamStatements = new ArrayList<>(); Ddl.Builder mergedDdl = informationSchemaDdl.toBuilder(); List ddlStatements = new ArrayList<>(); @@ -482,6 +504,19 @@ public void processElement(ProcessContext c) { c.output(pendingForeignKeysTag, createForeignKeyStatements); } + if (!missingChangeStreams.isEmpty()) { + Ddl.Builder builder = Ddl.builder(); + for (KV kv : missingChangeStreams) { + ChangeStream changeStream = + converter.toChangeStream(kv.getKey(), kv.getValue()); + builder.addChangeStream(changeStream); + } + Ddl newDdl = builder.build(); + createChangeStreamStatements.addAll( + newDdl.createChangeStreamStatements()); + } + c.output(pendingChangeStreamsTag, createChangeStreamStatements); + if (!ddlStatements.isEmpty()) { DatabaseAdminClient databaseAdminClient = spannerAccessor.getDatabaseAdminClient(); @@ -511,7 +546,10 @@ public void processElement(ProcessContext c) { }) .withSideInputs(avroSchemasView, informationSchemaView, manifestView) .withOutputTags( - ddlObjectTag, TupleTagList.of(pendingIndexesTag).and(pendingForeignKeysTag))); + ddlObjectTag, + TupleTagList.of(pendingIndexesTag) + .and(pendingForeignKeysTag) + .and(pendingChangeStreamsTag))); } } @@ -584,6 +622,12 @@ public void processElement(ProcessContext c) { c.output(KV.of(table.getName(), fullPath)); } } + for (Export.Table changeStream : proto.getChangeStreamsList()) { + for (String f : changeStream.getDataFilesList()) { + String fullPath = GcsUtil.joinPath(importDirectory.get(), f); + c.output(KV.of(changeStream.getName(), fullPath)); + } + } } })); @@ -601,6 +645,11 @@ public void processElement(ProcessContext c) { c.output(KV.of(table.getName(), table.getManifestFile())); } } + for (Export.Table changeStream : proto.getChangeStreamsList()) { + if (!Strings.isNullOrEmpty(changeStream.getManifestFile())) { + c.output(KV.of(changeStream.getName(), changeStream.getManifestFile())); + } + } } })); diff --git a/src/main/proto/export.proto b/src/main/proto/export.proto index 43c65046c9..6a16653210 100644 --- a/src/main/proto/export.proto +++ b/src/main/proto/export.proto @@ -20,12 +20,12 @@ message TableManifest { // Protobuf schema for the content of the export summary file representing the // whole database. message Export { - // An export summary of the table. - // During the import, `name` will be used as the Cloud Spanner table name - // and both `data_files` and `manifest_file` can be used to locate the - // corresponding data files. + // An export summary of the table, view or change stream. + // During the import, `name` will be used as the Cloud Spanner table, view or + // change stream name. Both `data_files` and `manifest_file` can be used to + // locate the corresponding data files. message Table { - // The name of the table. + // The name of the table, view or change stream. string name = 1; // Set of Avro file globs. // Use `data_files` when table manifest files with checksums are not @@ -45,4 +45,6 @@ message Export { // The set of tables that were exported. repeated Table tables = 1; repeated DatabaseOption database_options = 2; + // The set of change streams that were exported. + repeated Table change_streams = 3; } diff --git a/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java b/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java index ff35f9704f..4f6b5de4e6 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java @@ -318,6 +318,45 @@ public void checkConstraints() throws Exception { runTest(); } + @Test + public void changeStreams() throws Exception { + Ddl ddl = + Ddl.builder() + .createTable("T1") + .endTable() + .createTable("T2") + .column("key") + .int64() + .endColumn() + .column("c1") + .int64() + .endColumn() + .column("c2") + .string() + .max() + .endColumn() + .primaryKey() + .asc("key") + .end() + .endTable() + .createTable("T3") + .endTable() + .createChangeStream("ChangeStreamAll") + .forClause("FOR ALL") + .options( + ImmutableList.of( + "retention_period=\"7d\"", "value_capture_type=\"OLD_AND_NEW_VALUES\"")) + .endChangeStream() + .createChangeStream("ChangeStreamEmpty") + .endChangeStream() + .createChangeStream("ChangeStreamTableColumns") + .forClause("FOR `T1`, `T2`(`c1`, `c2`), `T3`()") + .endChangeStream() + .build(); + createAndPopulate(ddl, 0); + runTest(); + } + @Test public void randomSchema() throws Exception { Ddl ddl = RandomDdlGenerator.builder().build().generate(); @@ -354,6 +393,7 @@ private void runTest() { ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(30))); PipelineResult importResult = importPipeline.run(); importResult.waitUntilFinish(); diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java index e7f3cd10e4..7d6c349462 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportRelatedTablesCheckTest.java @@ -1035,6 +1035,7 @@ private void exportAndImportDb( ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(30))); PipelineResult importResult = importPipeline.run(); importResult.waitUntilFinish(); diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java index bd7c842334..744798dae4 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java @@ -302,6 +302,7 @@ private void exportAndImportDbAtTime( ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(true), ValueProvider.StaticValueProvider.of(30))); PipelineResult importResult = importPipeline.run(); importResult.waitUntilFinish(); diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java index 75f10532f3..00ff8051d3 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java @@ -100,7 +100,13 @@ public void buildTableManifests() throws Exception { @Test public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { Map tablesAndManifests = - ImmutableMap.of("table1", "table1 manifest", "table2", "table2 manifest"); + ImmutableMap.of( + "table1", + "table1 manifest", + "table2", + "table2 manifest", + "changeStream", + "changeStream manifest"); PCollection> metadataTables = pipeline @@ -115,6 +121,7 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { .build()); Ddl.Builder ddlBuilder = Ddl.builder(); ddlBuilder.mergeDatabaseOptions(databaseOptions); + ddlBuilder.createChangeStream("changeStream").endChangeStream(); Ddl ddl = ddlBuilder.build(); PCollectionView ddlView = pipeline.apply(Create.of(ddl)).apply(View.asSingleton()); PCollection databaseManifest = @@ -148,6 +155,12 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { String optionValue = dbOptions.getOptionValue(); assertThat(optionName, is("version_retention_period")); assertThat(optionValue, is("5d")); + + assertThat(manifestProto.getChangeStreamsCount(), is(1)); + assertThat(manifestProto.getChangeStreams(0).getName(), is("changeStream")); + assertThat( + manifestProto.getChangeStreams(0).getManifestFile(), + is("changeStream-manifest.json")); return null; }); diff --git a/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java b/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java index 251907acdc..3a8e3f1542 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner; +import static org.hamcrest.Matchers.equalToCompressingWhiteSpace; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -27,6 +28,9 @@ import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; @@ -422,6 +426,114 @@ public void generatedColumns() throws Exception { .build())); } + @Test + public void changeStreams() throws Exception { + Map avroFiles = new HashMap<>(); + avroFiles.put( + "ChangeStreamAll.avro", + SchemaBuilder.record("ChangeStreamAll") + .prop("spannerChangeStreamForClause", "FOR ALL") + .prop("spannerOption_0", "retention_period=\"7d\"") + .prop("spannerOption_1", "value_capture_type=\"OLD_AND_NEW_VALUES\"") + .fields() + .endRecord()); + avroFiles.put( + "ChangeStreamEmpty.avro", + SchemaBuilder.record("ChangeStreamEmpty") + .prop("spannerChangeStreamForClause", "") + .fields() + .endRecord()); + avroFiles.put( + "ChangeStreamTable.avro", + SchemaBuilder.record("ChangeStreamTable") + .prop("spannerChangeStreamForClause", "FOR T") + .fields() + .endRecord()); + avroFiles.put( + "ChangeStreamColumns.avro", + SchemaBuilder.record("ChangeStreamColumns") + .prop("spannerChangeStreamForClause", "FOR T(c1, c2)") + .fields() + .endRecord()); + avroFiles.put( + "ChangeStreamKeyOnly.avro", + SchemaBuilder.record("ChangeStreamKeyOnly") + .prop("spannerChangeStreamForClause", "FOR T()") + .fields() + .endRecord()); + + ExportProtos.Export.Builder exportProtoBuilder = ExportProtos.Export.newBuilder(); + for (Entry entry : avroFiles.entrySet()) { + String fileName = entry.getKey(); + Schema schema = entry.getValue(); + exportProtoBuilder.addChangeStreams( + ExportProtos.Export.Table.newBuilder() + .setName(schema.getName()) + .addDataFiles(fileName) + .build()); + // Create the Avro files to be imported. + File avroFile = tmpDir.newFile(fileName); + try (DataFileWriter fileWriter = + new DataFileWriter<>(new GenericDatumWriter<>(schema))) { + fileWriter.create(schema, avroFile); + } + } + + // Create the database manifest file. + ExportProtos.Export exportProto = exportProtoBuilder.build(); + File manifestFile = tmpDir.newFile("spanner-export.json"); + String manifestFileLocation = manifestFile.getParent(); + Files.write( + manifestFile.toPath(), + JsonFormat.printer().print(exportProto).getBytes(StandardCharsets.UTF_8)); + + // Create the target database. + String spannerSchema = + "CREATE TABLE `T` (" + + "`id` INT64 NOT NULL," + + "`c1` BOOL," + + "`c2` INT64," + + ") PRIMARY KEY (`id`)"; + spannerServer.createDatabase(dbName, Collections.singleton(spannerSchema)); + + // Run the import pipeline. + importPipeline.apply( + "Import", + new ImportTransform( + spannerServer.getSpannerConfig(dbName), + ValueProvider.StaticValueProvider.of(manifestFileLocation), + ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(true), + ValueProvider.StaticValueProvider.of(30))); + PipelineResult importResult = importPipeline.run(); + importResult.waitUntilFinish(); + + Ddl ddl; + try (ReadOnlyTransaction ctx = spannerServer.getDbClient(dbName).readOnlyTransaction()) { + ddl = new InformationSchemaScanner(ctx).scan(); + } + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE TABLE `T` (" + + " `id` INT64 NOT NULL," + + " `c1` BOOL," + + " `c2` INT64," + + " ) PRIMARY KEY (`id` ASC)" + + " CREATE CHANGE STREAM `ChangeStreamAll`" + + " FOR ALL" + + " OPTIONS (retention_period=\"7d\", value_capture_type=\"OLD_AND_NEW_VALUES\")" + + " CREATE CHANGE STREAM `ChangeStreamColumns`" + + " FOR `T`(`c1`, `c2`)" + + " CREATE CHANGE STREAM `ChangeStreamEmpty`" + + " CREATE CHANGE STREAM `ChangeStreamKeyOnly`" + + " FOR `T`()" + + " CREATE CHANGE STREAM `ChangeStreamTable`" + + " FOR `T`")); + } + private void runTest(Schema avroSchema, String spannerSchema, Iterable records) throws Exception { // Create the Avro file to be imported. @@ -468,6 +580,7 @@ private void runTest(Schema avroSchema, String spannerSchema, Iterable Date: Tue, 10 May 2022 16:31:10 -0700 Subject: [PATCH 132/145] Update on schemaConverters for Cloud Spanner PostgreSQL interface PiperOrigin-RevId: 447855961 --- .../spanner/AvroSchemaToDdlConverter.java | 104 ++++- .../spanner/DdlToAvroSchemaConverter.java | 48 ++- .../spanner/AvroSchemaToDdlConverterTest.java | 170 ++++++++ .../teleport/spanner/CopySchemaTest.java | 25 ++ .../spanner/DdlToAvroSchemaConverterTest.java | 387 ++++++++++++++++++ .../spanner/ddl/RandomDdlGenerator.java | 174 ++++++-- 6 files changed, 853 insertions(+), 55 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java index 64fffeafc6..cfd280c79c 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java @@ -22,6 +22,7 @@ import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.ddl.View; import com.google.common.base.Strings; @@ -36,9 +37,18 @@ /** Allows to convert a set of Avro schemas to {@link Ddl}. */ public class AvroSchemaToDdlConverter { private static final Logger LOG = LoggerFactory.getLogger(AvroSchemaToDdlConverter.class); + private final Dialect dialect; + + public AvroSchemaToDdlConverter() { + this.dialect = Dialect.GOOGLE_STANDARD_SQL; + } + + public AvroSchemaToDdlConverter(Dialect dialect) { + this.dialect = dialect; + } public Ddl toDdl(Collection avroSchemas) { - Ddl.Builder builder = Ddl.builder(); + Ddl.Builder builder = Ddl.builder(dialect); for (Schema schema : avroSchemas) { if (schema.getProp("spannerViewQuery") != null) { builder.addView(toView(null, schema)); @@ -57,7 +67,8 @@ public View toView(String viewName, Schema schema) { } LOG.debug("Converting to Ddl viewName {}", viewName); - View.Builder builder = View.builder().name(viewName).query(schema.getProp("spannerViewQuery")); + View.Builder builder = + View.builder(dialect).name(viewName).query(schema.getProp("spannerViewQuery")); if (schema.getProp("spannerViewSecurity") != null) { builder.security(View.SqlSecurity.valueOf(schema.getProp("spannerViewSecurity"))); } @@ -94,7 +105,7 @@ public Table toTable(String tableName, Schema schema) { } LOG.debug("Converting to Ddl tableName {}", tableName); - Table.Builder table = Table.builder(); + Table.Builder table = Table.builder(dialect); table.name(tableName); for (Schema.Field f : schema.getFields()) { Column.Builder column = table.column(f.name()); @@ -134,7 +145,8 @@ public Table toTable(String tableName, Schema schema) { Type spannerType = inferType(avroType, true); sqlType = toString(spannerType, true); } - column.parseType(sqlType).notNull(!nullable); + String defaultExpression = f.getProp("defaultExpression"); + column.parseType(sqlType).notNull(!nullable).defaultExpression(defaultExpression); } ImmutableList.Builder columnOptions = ImmutableList.builder(); for (int i = 0; ; i++) { @@ -155,10 +167,10 @@ public Table toTable(String tableName, Schema schema) { } if (spannerPrimaryKey.endsWith(" ASC")) { String name = spannerPrimaryKey.substring(0, spannerPrimaryKey.length() - 4); - table.primaryKey().asc(unescape(name)).end(); + table.primaryKey().asc(unescape(name, dialect)).end(); } else if (spannerPrimaryKey.endsWith(" DESC")) { String name = spannerPrimaryKey.substring(0, spannerPrimaryKey.length() - 5); - table.primaryKey().desc(unescape(name)).end(); + table.primaryKey().desc(unescape(name, dialect)).end(); } else { throw new IllegalArgumentException("Cannot parse spannerPrimaryKey " + spannerPrimaryKey); } @@ -194,8 +206,9 @@ public Table toTable(String tableName, Schema schema) { } // TODO: maybe encapsulate in the Ddl library. - private static String unescape(String name) { - if (name.startsWith("`") && name.endsWith("`")) { + private static String unescape(String name, Dialect dialect) { + if ((dialect == Dialect.GOOGLE_STANDARD_SQL && name.startsWith("`") && name.endsWith("`")) + || (dialect == Dialect.POSTGRESQL && name.startsWith("\"") && name.endsWith("\""))) { return name.substring(1, name.length() - 1); } return name; @@ -219,27 +232,44 @@ private com.google.cloud.teleport.spanner.common.Type inferType(Schema f, boolea switch (type) { case BOOLEAN: - return com.google.cloud.teleport.spanner.common.Type.bool(); + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.bool() + : com.google.cloud.teleport.spanner.common.Type.pgBool(); case INT: - return com.google.cloud.teleport.spanner.common.Type.int64(); + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.int64() + : com.google.cloud.teleport.spanner.common.Type.pgInt8(); case LONG: - if (LogicalTypes.timestampMillis().equals(logicalType)) { - return com.google.cloud.teleport.spanner.common.Type.timestamp(); + if (LogicalTypes.timestampMillis().equals(logicalType) + || LogicalTypes.timestampMicros().equals(logicalType)) { + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.timestamp() + : com.google.cloud.teleport.spanner.common.Type.pgTimestamptz(); } - if (LogicalTypes.timestampMicros().equals(logicalType)) { - return com.google.cloud.teleport.spanner.common.Type.timestamp(); - } - return com.google.cloud.teleport.spanner.common.Type.int64(); + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.int64() + : com.google.cloud.teleport.spanner.common.Type.pgInt8(); case FLOAT: case DOUBLE: - return com.google.cloud.teleport.spanner.common.Type.float64(); + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.float64() + : com.google.cloud.teleport.spanner.common.Type.pgFloat8(); case STRING: - return com.google.cloud.teleport.spanner.common.Type.string(); + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.string() + : com.google.cloud.teleport.spanner.common.Type.pgVarchar(); case BYTES: - if (LogicalTypes.decimal(NumericUtils.PRECISION, NumericUtils.SCALE).equals(logicalType)) { + if (LogicalTypes.decimal(NumericUtils.PRECISION, NumericUtils.SCALE).equals(logicalType) + && dialect == Dialect.GOOGLE_STANDARD_SQL) { return com.google.cloud.teleport.spanner.common.Type.numeric(); } - return com.google.cloud.teleport.spanner.common.Type.bytes(); + if (LogicalTypes.decimal(NumericUtils.PG_MAX_PRECISION, NumericUtils.PG_MAX_SCALE) + .equals(logicalType) && dialect == Dialect.POSTGRESQL) { + return com.google.cloud.teleport.spanner.common.Type.pgNumeric(); + } + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.bytes() + : com.google.cloud.teleport.spanner.common.Type.pgBytea(); case ARRAY: { if (supportArrays) { @@ -252,7 +282,10 @@ private com.google.cloud.teleport.spanner.common.Type inferType(Schema f, boolea element = unpacked; } try { - return com.google.cloud.teleport.spanner.common.Type.array(inferType(element, false)); + return (dialect == Dialect.GOOGLE_STANDARD_SQL) + ? com.google.cloud.teleport.spanner.common.Type.array(inferType(element, false)) + : com.google.cloud.teleport.spanner.common.Type.pgArray( + inferType(element, false)); } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Cannot infer array type for field " + f); } @@ -269,20 +302,38 @@ private String toString( switch (spannerType.getCode()) { case BOOL: return "BOOL"; + case PG_BOOL: + return "boolean"; case INT64: return "INT64"; + case PG_INT8: + return "bigint"; case FLOAT64: return "FLOAT64"; + case PG_FLOAT8: + return "double precision"; case STRING: return "STRING(MAX)"; + case PG_TEXT: + return "text"; + case PG_VARCHAR: + return "character varying"; case BYTES: return "BYTES(MAX)"; + case PG_BYTEA: + return "bytea"; case TIMESTAMP: return "TIMESTAMP"; + case PG_TIMESTAMPTZ: + return "timestamp with time zone"; case DATE: return "DATE"; + case PG_DATE: + return "date"; case NUMERIC: return "NUMERIC"; + case PG_NUMERIC: + return "numeric"; case JSON: return "JSON"; case ARRAY: @@ -296,6 +347,17 @@ private String toString( // otherwise fall through and throw an error. break; } + case PG_ARRAY: + { + if (supportArray) { + com.google.cloud.teleport.spanner.common.Type element = + spannerType.getArrayElementType(); + String elementStr = toString(element, false); + return elementStr + "[]"; + } + // otherwise fall through and throw an error. + break; + } } throw new IllegalArgumentException("Cannot to string the type " + spannerType); } diff --git a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java index ec2da1eddb..50ce35ece0 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java @@ -19,6 +19,7 @@ import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.IndexColumn; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.ddl.View; @@ -55,15 +56,29 @@ public Collection convert(Ddl ddl) { recordBuilder.prop( "spannerOnDeleteAction", table.onDeleteCascade() ? "cascade" : "no action"); } - if (table.primaryKeys() != null) { - String encodedPk = - table.primaryKeys().stream() - .map(IndexColumn::prettyPrint) - .collect(Collectors.joining(",")); - recordBuilder.prop("spannerPrimaryKey", encodedPk); - } - for (int i = 0; i < table.primaryKeys().size(); i++) { - recordBuilder.prop("spannerPrimaryKey_" + i, table.primaryKeys().get(i).prettyPrint()); + if (table.dialect() == Dialect.GOOGLE_STANDARD_SQL) { + if (table.primaryKeys() != null) { + String encodedPk = + table.primaryKeys().stream() + .map(IndexColumn::prettyPrint) + .collect(Collectors.joining(",")); + recordBuilder.prop("spannerPrimaryKey", encodedPk); + } + for (int i = 0; i < table.primaryKeys().size(); i++) { + recordBuilder.prop("spannerPrimaryKey_" + i, table.primaryKeys().get(i).prettyPrint()); + } + } else if (table.dialect() == Dialect.POSTGRESQL) { + if (table.primaryKeys() != null) { + String encodedPk = + table.primaryKeys().stream() + .map(c -> "\"" + c.name() + "\"") + .collect(Collectors.joining(", ")); + recordBuilder.prop("spannerPrimaryKey", encodedPk); + } + for (int i = 0; i < table.primaryKeys().size(); i++) { + IndexColumn pk = table.primaryKeys().get(i); + recordBuilder.prop("spannerPrimaryKey_" + i, "\"" + pk.name() + "\" ASC"); + } } for (int i = 0; i < table.indexes().size(); i++) { recordBuilder.prop("spannerIndex_" + i, table.indexes().get(i)); @@ -89,6 +104,9 @@ public Collection convert(Ddl ddl) { // which are semantically logical entities. fieldBuilder.type(SchemaBuilder.builder().nullType()).withDefault(null); } else { + if (cm.defaultExpression() != null) { + fieldBuilder.prop("defaultExpression", cm.defaultExpression()); + } Schema avroType = avroType(cm.type()); if (!cm.notNull()) { avroType = wrapAsNullable(avroType); @@ -134,25 +152,37 @@ public Collection convert(Ddl ddl) { private Schema avroType(com.google.cloud.teleport.spanner.common.Type spannerType) { switch (spannerType.getCode()) { case BOOL: + case PG_BOOL: return SchemaBuilder.builder().booleanType(); case INT64: + case PG_INT8: return SchemaBuilder.builder().longType(); case FLOAT64: + case PG_FLOAT8: return SchemaBuilder.builder().doubleType(); + case PG_TEXT: + case PG_VARCHAR: case STRING: case DATE: + case PG_DATE: case JSON: return SchemaBuilder.builder().stringType(); case BYTES: + case PG_BYTEA: return SchemaBuilder.builder().bytesType(); case TIMESTAMP: + case PG_TIMESTAMPTZ: return shouldExportTimestampAsLogicalType ? LogicalTypes.timestampMicros().addToSchema(SchemaBuilder.builder().longType()) : SchemaBuilder.builder().stringType(); case NUMERIC: return LogicalTypes.decimal(NumericUtils.PRECISION, NumericUtils.SCALE) .addToSchema(SchemaBuilder.builder().bytesType()); + case PG_NUMERIC: + return LogicalTypes.decimal(NumericUtils.PG_MAX_PRECISION, NumericUtils.PG_MAX_SCALE) + .addToSchema(SchemaBuilder.builder().bytesType()); case ARRAY: + case PG_ARRAY: Schema avroItemsType = avroType(spannerType.getArrayElementType()); return SchemaBuilder.builder().array().items().type(wrapAsNullable(avroItemsType)); default: diff --git a/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java index 07ffb61913..5ba2eef970 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java @@ -18,9 +18,11 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.text.IsEqualCompressingWhiteSpace.equalToCompressingWhiteSpace; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -37,6 +39,14 @@ public void emptySchema() { assertThat(ddl.allTables(), empty()); } + @Test + public void pgEmptySchema() { + AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(Dialect.POSTGRESQL); + Ddl ddl = converter.toDdl(Collections.emptyList()); + assertEquals(ddl.dialect(), Dialect.POSTGRESQL); + assertThat(ddl.allTables(), empty()); + } + @Test public void simple() { String avroString = @@ -149,6 +159,139 @@ public void simple() { + " FOREIGN KEY (`first_name`) REFERENCES `AllowedNames` (`first_name`)")); } + @Test + public void pgSimple() { + String avroString = + "{" + + " \"type\" : \"record\"," + + " \"name\" : \"Users\"," + + " \"namespace\" : \"spannertest\"," + + " \"fields\" : [ {" + + " \"name\" : \"id\"," + + " \"type\" : \"long\"," + + " \"sqlType\" : \"bigint\"" + + " }, {" + + " \"name\" : \"first_name\"," + + " \"type\" : [ \"null\", \"string\" ]," + + " \"sqlType\" : \"character varying(10)\"," + + " \"defaultExpression\" : \"'John'\"" + + " }, {" + + " \"name\" : \"last_name\"," + + " \"type\" : [ \"null\", \"string\" ]," + + " \"sqlType\" : \"character varying\"" + + " }, {" + + " \"name\" : \"full_name\"," + + " \"type\" : \"null\"," + + " \"sqlType\" : \"character varying\"," + + " \"notNull\" : \"false\"," + + " \"generationExpression\" : \"CONCAT(first_name, ' ', last_name)\"," + + " \"stored\" : \"true\"" + + " }, {" + + " \"name\" : \"numeric\"," + + " \"type\" : [\"null\", {\"type\":\"bytes\",\"logicalType\":\"decimal\"}]," + + " \"sqlType\" : \"numeric\"" + + " }, {" + + " \"name\" : \"numeric2\"," + + " \"type\" : [\"null\", {\"type\":\"bytes\",\"logicalType\":\"decimal\"," + + " \"precision\":147455,\"scale\":16383}]" // Omitting sqlType + + " }, {" + + " \"name\" : \"notNumeric\"," + + " \"type\" : [\"null\", {\"type\":\"bytes\",\"logicalType\":\"decimal\"," + + " \"precision\":147455}]" // Omitting sqlType + + " }, {" + + " \"name\" : \"notNumeric2\"," + + " \"type\" : [\"null\", {\"type\":\"bytes\",\"logicalType\":\"decimal\"," + + " \"precision\":147455,\"scale\":16384}]" // Omitting sqlType + + " }, {" + + " \"name\":\"numericArr\"," + + " \"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",{\"type\":\"bytes\"," + + " \"logicalType\":\"decimal\",\"precision\":147455,\"scale\":16383}]}]" + // Omitting sqlType + + " }, {" + + " \"name\":\"notNumericArr\"," + + " \"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",{\"type\":\"bytes\"," + + " \"logicalType\":\"decimal\",\"precision\":147455}]}]" // Omitting sqlType + + " }, {" + + " \"name\" : \"bool\"," + + " \"type\" : [ \"null\", \"boolean\" ]," + + " \"sqlType\" : \"boolean\"" + + " }, {" + + " \"name\" : \"float\"," + + " \"type\" : [ \"null\", \"double\" ]," + + " \"sqlType\" : \"double precision\"" + + " }, {" + + " \"name\" : \"bytes\"," + + " \"type\" : [ \"null\", \"bytes\" ]," + + " \"sqlType\" : \"bytea\"" + + " }, {" + + " \"name\" : \"text\"," + + " \"type\" : [ \"null\", \"string\" ]," + + " \"sqlType\" : \"text\"" + + " }, {" + + " \"name\" : \"timestamptz\"," + + " \"type\" : [ \"null\", \"string\" ]," + + " \"sqlType\" : \"timestamp with time zone\"" + + " }, {" + + " \"name\" : \"date\"," + + " \"type\" : [ \"null\", \"string\" ]," + + " \"sqlType\" : \"date\"" + + " }, {" + + " \"name\" : \"varcharArr1\"," + + " \"type\" : [\"null\"," + + " {\"type\":\"array\",\"items\":[\"null\",{\"type\":\"string\"}]}]," + + " \"sqlType\" : \"character varying[]\"" + + " }, {" + + " \"name\" : \"varcharArr2\"," + + " \"type\" : [\"null\"," + + " {\"type\":\"array\",\"items\":[\"null\",{\"type\":\"string\"}]}]" + // Omitting sqlType + + " } ], \"googleStorage\" : \"CloudSpanner\", \"spannerParent\" : \"\", " + + " \"googleFormatVersion\" : \"booleans\", \"spannerPrimaryKey_0\" : \"\\\"id\\\"" + + " ASC\", \"spannerPrimaryKey_1\" : \"\\\"last_name\\\" ASC\", \"spannerIndex_0\" :" + + " \"CREATE INDEX \\\"UsersByFirstName\\\" ON \\\"Users\\\" (\\\"first_name\\\")\", " + + " \"spannerForeignKey_0\" : \"ALTER TABLE \\\"Users\\\" ADD CONSTRAINT \\\"fk\\\"" + + " FOREIGN KEY (\\\"first_name\\\") REFERENCES \\\"AllowedNames\\\"" + + " (\\\"first_name\\\")\", \"spannerCheckConstraint_0\" : \"CONSTRAINT \\\"ck\\\"" + + " CHECK(\\\"first_name\\\" != \\\"last_name\\\")\"}"; + + Schema schema = new Schema.Parser().parse(avroString); + + AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(Dialect.POSTGRESQL); + Ddl ddl = converter.toDdl(Collections.singleton(schema)); + assertEquals(ddl.dialect(), Dialect.POSTGRESQL); + assertThat(ddl.allTables(), hasSize(1)); + assertThat(ddl.views(), hasSize(0)); + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE TABLE \"Users\" (" + + " \"id\" bigint NOT NULL," + + " \"first_name\" character varying(10) DEFAULT 'John'," + + " \"last_name\" character varying," + + " \"full_name\" character varying GENERATED ALWAYS AS" + + " (CONCAT(first_name, ' ', last_name)) STORED," + + " \"numeric\" numeric," + + " \"numeric2\" numeric," + + " \"notNumeric\" bytea," + + " \"notNumeric2\" bytea," + + " \"numericArr\" numeric[]," + + " \"notNumericArr\" bytea[]," + + " \"bool\" boolean," + + " \"float\" double precision," + + " \"bytes\" bytea," + + " \"text\" text," + + " \"timestamptz\" timestamp with time zone," + + " \"date\" date," + + " \"varcharArr1\" character varying[]," + + " \"varcharArr2\" character varying[]," + + " CONSTRAINT \"ck\" CHECK(\"first_name\" != \"last_name\")," + + " PRIMARY KEY (\"id\", \"last_name\")" + + " )" + + " CREATE INDEX \"UsersByFirstName\" ON \"Users\" (\"first_name\")" + + " ALTER TABLE \"Users\" ADD CONSTRAINT \"fk\" FOREIGN KEY (\"first_name\")" + + " REFERENCES \"AllowedNames\" (\"first_name\")")); + } + @Test public void invokerRightsView() { String avroString = @@ -174,6 +317,33 @@ public void invokerRightsView() { "CREATE VIEW `Names` SQL SECURITY INVOKER AS SELECT first_name, last_name FROM Users")); } + @Test + public void pgInvokerRightsView() { + String avroString = + "{" + + " \"type\" : \"record\"," + + " \"name\" : \"Names\"," + + " \"fields\" : []," + + " \"namespace\" : \"spannertest\"," + + " \"googleStorage\" : \"CloudSpanner\"," + + " \"googleFormatVersion\" : \"booleans\"," + + " \"spannerViewSecurity\" : \"INVOKER\"," + + " \"spannerViewQuery\" : \"SELECT first_name, last_name FROM Users\"" + + "}"; + + Schema schema = new Schema.Parser().parse(avroString); + + AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(Dialect.POSTGRESQL); + Ddl ddl = converter.toDdl(Collections.singleton(schema)); + assertEquals(ddl.dialect(), Dialect.POSTGRESQL); + assertThat(ddl.views(), hasSize(1)); + assertThat( + ddl.prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE VIEW \"Names\" SQL SECURITY INVOKER AS SELECT first_name, last_name FROM" + + " Users")); + } + @Test public void columnOptions() { String avroString = diff --git a/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java b/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java index 36034e55fc..306c916c30 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertThat; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.RandomDdlGenerator; import java.io.IOException; import java.util.Collection; @@ -54,4 +55,28 @@ public void copyRandomSchema() { assertThat(copied.prettyPrint(), equalToCompressingWhiteSpace(ddl.prettyPrint())); } + + @Test + public void pgCopyRandomSchema() { + Ddl ddl = + RandomDdlGenerator.builder(Dialect.POSTGRESQL) + .setMaxPkComponents(2) + .setMaxBranchPerLevel(new int[] {5, 4, 3, 2, 2, 3, 3}) + .setMaxViews(3) + .build() + .generate(); + try { + ddl.prettyPrint(System.out); + } catch (IOException e) { + throw new RuntimeException(e); + } + + DdlToAvroSchemaConverter ddlToAvro = new DdlToAvroSchemaConverter("spanner", "test", false); + AvroSchemaToDdlConverter avroToDdl = new AvroSchemaToDdlConverter(Dialect.POSTGRESQL); + + Collection schemas = ddlToAvro.convert(ddl); + Ddl copied = avroToDdl.toDdl(schemas); + + assertThat(copied.prettyPrint(), equalToCompressingWhiteSpace(ddl.prettyPrint())); + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java index a17add8f46..0c1b615c6d 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java @@ -25,6 +25,7 @@ import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.View; import com.google.common.collect.ImmutableList; import java.util.Collection; @@ -45,6 +46,14 @@ public void emptyDb() { assertThat(converter.convert(empty), empty()); } + @Test + public void pgEmptyDb() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", false); + Ddl empty = Ddl.builder(Dialect.POSTGRESQL).build(); + assertThat(converter.convert(empty), empty()); + } + @Test public void simple() { DdlToAvroSchemaConverter converter = @@ -149,6 +158,114 @@ public void simple() { System.out.println(avroSchema.toString(true)); } + @Test + public void pgSimple() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", false); + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("first_name") + .pgVarchar() + .size(10) + .defaultExpression("'John'") + .endColumn() + .column("last_name") + .type(Type.pgVarchar()) + .max() + .endColumn() + .column("full_name") + .type(Type.pgVarchar()) + .max() + .generatedAs("CONCAT(first_name, ' ', last_name)") + .stored() + .endColumn() + .primaryKey() + .asc("id") + .asc("last_name") + .end() + .indexes( + ImmutableList.of("CREATE INDEX \"UsersByFirstName\" ON \"Users\" (\"first_name\")")) + .foreignKeys( + ImmutableList.of( + "ALTER TABLE \"Users\" ADD CONSTRAINT \"fk\" FOREIGN KEY (\"first_name\")" + + " REFERENCES \"AllowedNames\" (\"first_name\")")) + .checkConstraints( + ImmutableList.of("CONSTRAINT ck CHECK (\"first_name\" != \"last_name\")")) + .endTable() + .build(); + + Collection result = converter.convert(ddl); + assertThat(result, hasSize(1)); + Schema avroSchema = result.iterator().next(); + + assertThat(avroSchema.getNamespace(), equalTo("spannertest")); + assertThat(avroSchema.getProp("googleFormatVersion"), equalTo("booleans")); + assertThat(avroSchema.getProp("googleStorage"), equalTo("CloudSpanner")); + + assertThat(avroSchema.getName(), equalTo("Users")); + + List fields = avroSchema.getFields(); + + assertThat(fields, hasSize(4)); + + assertThat(fields.get(0).name(), equalTo("id")); + // Not null + assertThat(fields.get(0).schema().getType(), equalTo(Schema.Type.LONG)); + assertThat(fields.get(0).getProp("sqlType"), equalTo("bigint")); + assertThat(fields.get(0).getProp("notNull"), equalTo(null)); + assertThat(fields.get(0).getProp("generationExpression"), equalTo(null)); + assertThat(fields.get(0).getProp("stored"), equalTo(null)); + + assertThat(fields.get(1).name(), equalTo("first_name")); + assertThat(fields.get(1).schema(), equalTo(nullableUnion(Schema.Type.STRING))); + assertThat(fields.get(1).getProp("sqlType"), equalTo("character varying(10)")); + assertThat(fields.get(1).getProp("notNull"), equalTo(null)); + assertThat(fields.get(1).getProp("generationExpression"), equalTo(null)); + assertThat(fields.get(1).getProp("stored"), equalTo(null)); + assertThat(fields.get(1).getProp("defaultExpression"), equalTo("'John'")); + + assertThat(fields.get(2).name(), equalTo("last_name")); + assertThat(fields.get(2).schema(), equalTo(nullableUnion(Schema.Type.STRING))); + assertThat(fields.get(2).getProp("sqlType"), equalTo("character varying")); + assertThat(fields.get(2).getProp("notNull"), equalTo(null)); + assertThat(fields.get(2).getProp("generationExpression"), equalTo(null)); + assertThat(fields.get(2).getProp("stored"), equalTo(null)); + assertThat(fields.get(2).getProp("defaultExpression"), equalTo(null)); + + assertThat(fields.get(3).name(), equalTo("full_name")); + assertThat(fields.get(3).schema(), equalTo(Schema.create(Schema.Type.NULL))); + assertThat(fields.get(3).getProp("sqlType"), equalTo("character varying")); + assertThat(fields.get(3).getProp("notNull"), equalTo("false")); + assertThat( + fields.get(3).getProp("generationExpression"), + equalTo("CONCAT(first_name, ' ', last_name)")); + assertThat(fields.get(3).getProp("stored"), equalTo("true")); + assertThat(fields.get(3).getProp("defaultExpression"), equalTo(null)); + + // spanner pk + assertThat(avroSchema.getProp("spannerPrimaryKey_0"), equalTo("\"id\" ASC")); + assertThat(avroSchema.getProp("spannerPrimaryKey_1"), equalTo("\"last_name\" ASC")); + assertThat(avroSchema.getProp("spannerParent"), nullValue()); + assertThat(avroSchema.getProp("spannerOnDeleteAction"), nullValue()); + + assertThat( + avroSchema.getProp("spannerIndex_0"), + equalTo("CREATE INDEX \"UsersByFirstName\" ON \"Users\" (\"first_name\")")); + assertThat( + avroSchema.getProp("spannerForeignKey_0"), + equalTo( + "ALTER TABLE \"Users\" ADD CONSTRAINT \"fk\" FOREIGN KEY (\"first_name\")" + + " REFERENCES \"AllowedNames\" (\"first_name\")")); + assertThat( + avroSchema.getProp("spannerCheckConstraint_0"), + equalTo("CONSTRAINT ck CHECK (\"first_name\" != \"last_name\")")); + } + @Test public void invokerRightsView() { DdlToAvroSchemaConverter converter = @@ -195,6 +312,52 @@ public void invokerRightsView() { assertThat(avroView.getName(), equalTo("Names")); } + @Test + public void pgInvokerRightsView() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", false); + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("first_name") + .pgVarchar() + .size(10) + .endColumn() + .column("last_name") + .type(Type.pgVarchar()) + .max() + .endColumn() + .endTable() + .createView("Names") + .query("SELECT first_name, last_name FROM Users") + .security(View.SqlSecurity.INVOKER) + .endView() + .build(); + + Collection result = converter.convert(ddl); + assertThat(result, hasSize(2)); + Schema avroView = null; + for (Schema s : result) { + if (s.getName().equals("Names")) { + avroView = s; + } + } + assertThat(avroView, notNullValue()); + + assertThat(avroView.getNamespace(), equalTo("spannertest")); + assertThat(avroView.getProp("googleFormatVersion"), equalTo("booleans")); + assertThat(avroView.getProp("googleStorage"), equalTo("CloudSpanner")); + assertThat( + avroView.getProp("spannerViewQuery"), equalTo("SELECT first_name, last_name FROM Users")); + assertThat(avroView.getProp("spannerViewSecurity"), equalTo("INVOKER")); + + assertThat(avroView.getName(), equalTo("Names")); + } + @Test public void allTypes() { DdlToAvroSchemaConverter converter = @@ -359,6 +522,166 @@ public void allTypes() { System.out.println(avroSchema.toString(true)); } + @Test + public void pgAllTypes() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", false); + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("AllTYPES") + .column("bool_field") + .pgBool() + .endColumn() + .column("int8_field") + .pgInt8() + .endColumn() + .column("float8_field") + .pgFloat8() + .endColumn() + .column("varchar_field") + .pgVarchar() + .max() + .endColumn() + .column("bytea_field") + .pgBytea() + .endColumn() + .column("timestamptz_field") + .pgTimestamptz() + .endColumn() + .column("numeric_field") + .pgNumeric() + .endColumn() + .column("text_field") + .pgText() + .endColumn() + .column("date_field") + .pgDate() + .endColumn() + .column("arr_bool_field") + .type(Type.pgArray(Type.pgBool())) + .endColumn() + .column("arr_int8_field") + .type(Type.pgArray(Type.pgInt8())) + .endColumn() + .column("arr_float8_field") + .type(Type.pgArray(Type.pgFloat8())) + .endColumn() + .column("arr_varchar_field") + .type(Type.pgArray(Type.pgVarchar())) + .max() + .endColumn() + .column("arr_bytea_field") + .type(Type.pgArray(Type.pgBytea())) + .endColumn() + .column("arr_timestamptz_field") + .type(Type.pgArray(Type.pgTimestamptz())) + .endColumn() + .column("arr_numeric_field") + .type(Type.pgArray(Type.pgNumeric())) + .endColumn() + .column("arr_text_field") + .type(Type.pgArray(Type.pgText())) + .endColumn() + .column("arr_date_field") + .type(Type.pgArray(Type.pgDate())) + .endColumn() + .primaryKey() + .asc("bool_field") + .end() + .interleaveInParent("ParentTable") + .onDeleteCascade() + .endTable() + .build(); + + Collection result = converter.convert(ddl); + assertThat(result, hasSize(1)); + Schema avroSchema = result.iterator().next(); + + assertThat(avroSchema.getNamespace(), equalTo("spannertest")); + assertThat(avroSchema.getProp("googleFormatVersion"), equalTo("booleans")); + assertThat(avroSchema.getProp("googleStorage"), equalTo("CloudSpanner")); + + List fields = avroSchema.getFields(); + + assertThat(fields, hasSize(18)); + + assertThat(fields.get(0).name(), equalTo("bool_field")); + assertThat(fields.get(0).schema(), equalTo(nullableUnion(Schema.Type.BOOLEAN))); + assertThat(fields.get(0).getProp("sqlType"), equalTo("boolean")); + + assertThat(fields.get(1).name(), equalTo("int8_field")); + assertThat(fields.get(1).schema(), equalTo(nullableUnion(Schema.Type.LONG))); + assertThat(fields.get(1).getProp("sqlType"), equalTo("bigint")); + + assertThat(fields.get(2).name(), equalTo("float8_field")); + assertThat(fields.get(2).schema(), equalTo(nullableUnion(Schema.Type.DOUBLE))); + assertThat(fields.get(2).getProp("sqlType"), equalTo("double precision")); + + assertThat(fields.get(3).name(), equalTo("varchar_field")); + assertThat(fields.get(3).schema(), equalTo(nullableUnion(Schema.Type.STRING))); + assertThat(fields.get(3).getProp("sqlType"), equalTo("character varying")); + + assertThat(fields.get(4).name(), equalTo("bytea_field")); + assertThat(fields.get(4).schema(), equalTo(nullableUnion(Schema.Type.BYTES))); + assertThat(fields.get(4).getProp("sqlType"), equalTo("bytea")); + + assertThat(fields.get(5).name(), equalTo("timestamptz_field")); + assertThat(fields.get(5).schema(), equalTo(nullableUnion(Schema.Type.STRING))); + assertThat(fields.get(5).getProp("sqlType"), equalTo("timestamp with time zone")); + + assertThat(fields.get(6).name(), equalTo("numeric_field")); + assertThat(fields.get(6).schema(), equalTo(nullablePgNumericUnion())); + assertThat(fields.get(6).getProp("sqlType"), equalTo("numeric")); + + assertThat(fields.get(7).name(), equalTo("text_field")); + assertThat(fields.get(7).schema(), equalTo(nullableUnion(Schema.Type.STRING))); + assertThat(fields.get(7).getProp("sqlType"), equalTo("text")); + + assertThat(fields.get(8).name(), equalTo("date_field")); + assertThat(fields.get(8).schema(), equalTo(nullableUnion(Schema.Type.STRING))); + assertThat(fields.get(8).getProp("sqlType"), equalTo("date")); + + assertThat(fields.get(9).name(), equalTo("arr_bool_field")); + assertThat(fields.get(9).schema(), equalTo(nullableArray(Schema.Type.BOOLEAN))); + assertThat(fields.get(9).getProp("sqlType"), equalTo("boolean[]")); + + assertThat(fields.get(10).name(), equalTo("arr_int8_field")); + assertThat(fields.get(10).schema(), equalTo(nullableArray(Schema.Type.LONG))); + assertThat(fields.get(10).getProp("sqlType"), equalTo("bigint[]")); + + assertThat(fields.get(11).name(), equalTo("arr_float8_field")); + assertThat(fields.get(11).schema(), equalTo(nullableArray(Schema.Type.DOUBLE))); + assertThat(fields.get(11).getProp("sqlType"), equalTo("double precision[]")); + + assertThat(fields.get(12).name(), equalTo("arr_varchar_field")); + assertThat(fields.get(12).schema(), equalTo(nullableArray(Schema.Type.STRING))); + assertThat(fields.get(12).getProp("sqlType"), equalTo("character varying[]")); + + assertThat(fields.get(13).name(), equalTo("arr_bytea_field")); + assertThat(fields.get(13).schema(), equalTo(nullableArray(Schema.Type.BYTES))); + assertThat(fields.get(13).getProp("sqlType"), equalTo("bytea[]")); + + assertThat(fields.get(14).name(), equalTo("arr_timestamptz_field")); + assertThat(fields.get(14).schema(), equalTo(nullableArray(Schema.Type.STRING))); + assertThat(fields.get(14).getProp("sqlType"), equalTo("timestamp with time zone[]")); + + assertThat(fields.get(15).name(), equalTo("arr_numeric_field")); + assertThat(fields.get(15).schema(), equalTo(nullablePgNumericArray())); + assertThat(fields.get(15).getProp("sqlType"), equalTo("numeric[]")); + + assertThat(fields.get(16).name(), equalTo("arr_text_field")); + assertThat(fields.get(16).schema(), equalTo(nullableArray(Schema.Type.STRING))); + assertThat(fields.get(16).getProp("sqlType"), equalTo("text[]")); + + assertThat(fields.get(17).name(), equalTo("arr_date_field")); + assertThat(fields.get(17).schema(), equalTo(nullableArray(Schema.Type.STRING))); + assertThat(fields.get(17).getProp("sqlType"), equalTo("date[]")); + + assertThat(avroSchema.getProp("spannerPrimaryKey_0"), equalTo("\"bool_field\" ASC")); + assertThat(avroSchema.getProp("spannerParent"), equalTo("ParentTable")); + assertThat(avroSchema.getProp("spannerOnDeleteAction"), equalTo("cascade")); + } + @Test public void timestampLogicalTypeTest() { DdlToAvroSchemaConverter converter = @@ -406,6 +729,53 @@ public void timestampLogicalTypeTest() { assertThat(fields.get(1).getProp("sqlType"), equalTo("TIMESTAMP")); } + @Test + public void pgTimestampLogicalTypeTest() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", true); + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("timestamp_field") + .pgTimestamptz() + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + + Collection result = converter.convert(ddl); + assertThat(result, hasSize(1)); + Schema avroSchema = result.iterator().next(); + + assertThat(avroSchema.getNamespace(), equalTo("spannertest")); + assertThat(avroSchema.getProp("googleFormatVersion"), equalTo("booleans")); + assertThat(avroSchema.getProp("googleStorage"), equalTo("CloudSpanner")); + + assertThat(avroSchema.getName(), equalTo("Users")); + + List fields = avroSchema.getFields(); + + assertThat(fields, hasSize(2)); + + assertThat(fields.get(0).name(), equalTo("id")); + // Not null + assertThat(fields.get(0).schema().getType(), equalTo(Schema.Type.LONG)); + assertThat(fields.get(0).getProp("sqlType"), equalTo("bigint")); + assertThat(fields.get(0).getProp("notNull"), equalTo(null)); + assertThat(fields.get(0).getProp("generationExpression"), equalTo(null)); + assertThat(fields.get(0).getProp("stored"), equalTo(null)); + + assertThat(fields.get(1).name(), equalTo("timestamp_field")); + assertThat(fields.get(1).schema(), equalTo(nullableTimestampUnion())); + assertThat(fields.get(1).getProp("sqlType"), equalTo("timestamp with time zone")); + } + @Test public void changeStreams() { DdlToAvroSchemaConverter converter = @@ -488,4 +858,21 @@ private Schema nullableNumericArray() { LogicalTypes.decimal(NumericUtils.PRECISION, NumericUtils.SCALE) .addToSchema(Schema.create(Schema.Type.BYTES))))); } + + private Schema nullablePgNumericArray() { + return Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.createArray( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.decimal(NumericUtils.PG_MAX_PRECISION, NumericUtils.PG_MAX_SCALE) + .addToSchema(Schema.create(Schema.Type.BYTES))))); + } + + private Schema nullablePgNumericUnion() { + return Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.decimal(NumericUtils.PG_MAX_PRECISION, NumericUtils.PG_MAX_SCALE) + .addToSchema(Schema.create(Schema.Type.BYTES))); + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java index e296cd998b..2a5d5b7a43 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java @@ -34,6 +34,18 @@ public abstract class RandomDdlGenerator { new Type.Code[] { Type.Code.BOOL, Type.Code.INT64, Type.Code.STRING, Type.Code.TIMESTAMP, Type.Code.DATE }; + + private static final Type.Code[] PG_PK_TYPES = + new Type.Code[] { + Type.Code.PG_BOOL, + Type.Code.PG_INT8, + Type.Code.PG_FLOAT8, + Type.Code.PG_TEXT, + Type.Code.PG_VARCHAR, + Type.Code.PG_TIMESTAMPTZ, + Type.Code.PG_DATE + }; + private static final Type.Code[] COLUMN_TYPES = new Type.Code[] { Type.Code.BOOL, @@ -44,6 +56,19 @@ public abstract class RandomDdlGenerator { Type.Code.TIMESTAMP, Type.Code.DATE }; + + private static final Type.Code[] PG_COLUMN_TYPES = + new Type.Code[] { + Type.Code.PG_BOOL, + Type.Code.PG_INT8, + Type.Code.PG_FLOAT8, + Type.Code.PG_VARCHAR, + Type.Code.PG_BYTEA, + Type.Code.PG_TIMESTAMPTZ, + Type.Code.PG_NUMERIC, + Type.Code.PG_DATE + }; + // Types that could be used by check constraint private static final Set CHECK_CONSTRAINT_TYPES = new HashSet<>( @@ -55,8 +80,22 @@ public abstract class RandomDdlGenerator { Type.Code.TIMESTAMP, Type.Code.DATE)); + private static final Set PG_CHECK_CONSTRAINT_TYPES = + new HashSet<>( + Arrays.asList( + Type.Code.PG_BOOL, + Type.Code.PG_INT8, + Type.Code.PG_FLOAT8, + Type.Code.PG_TEXT, + Type.Code.PG_VARCHAR, + Type.Code.PG_TIMESTAMPTZ, + Type.Code.PG_NUMERIC, + Type.Code.PG_DATE)); + private static final int MAX_PKS = 16; + public abstract Dialect getDialect(); + public abstract Random getRandom(); public abstract int getArrayChance(); @@ -80,8 +119,13 @@ public abstract class RandomDdlGenerator { public abstract int getMaxViews(); public static Builder builder() { + return builder(Dialect.GOOGLE_STANDARD_SQL); + } + + public static Builder builder(Dialect dialect) { return new AutoValue_RandomDdlGenerator.Builder() + .setDialect(dialect) .setRandom(new Random()) .setArrayChance(20) .setMaxPkComponents(3) @@ -90,18 +134,18 @@ public static Builder builder() { .setMaxViews(0) .setMaxIndex(2) .setMaxForeignKeys(2) - // TODO: enable once CHECK constraints are enabled - .setEnableCheckConstraints(false) + .setEnableCheckConstraints(true) .setMaxColumns(8) .setMaxIdLength(11) - // TODO: enable generated columns once they are supported. - .setEnableGeneratedColumns(false); + .setEnableGeneratedColumns(true); } /** A builder for {@link RandomDdlGenerator}. */ @AutoValue.Builder public abstract static class Builder { + public abstract Builder setDialect(Dialect dialect); + public abstract Builder setRandom(Random rnd); public abstract Builder setArrayChance(int chance); @@ -132,7 +176,7 @@ public abstract static class Builder { private Set allIdentifiers = Sets.newHashSet(); public Ddl generate() { - Ddl.Builder builder = Ddl.builder(); + Ddl.Builder builder = Ddl.builder(getDialect()); int numParentTables = 1 + getRandom().nextInt(getMaxBranchPerLevel()[0]); for (int i = 0; i < numParentTables; i++) { generateTable(builder, null, 0); @@ -160,8 +204,13 @@ private void generateView(Ddl.Builder builder) { if (!firstIncluded) { queryBuilder.append(", "); } + if (getDialect() == Dialect.POSTGRESQL) { + queryBuilder.append("\""); + } queryBuilder.append(column.name()); - + if (getDialect() == Dialect.POSTGRESQL) { + queryBuilder.append("\""); + } firstIncluded = false; } } @@ -169,8 +218,13 @@ private void generateView(Ddl.Builder builder) { queryBuilder.append("1"); } queryBuilder.append(" from "); + if (getDialect() == Dialect.POSTGRESQL) { + queryBuilder.append("\""); + } queryBuilder.append(sourceTable.name()); - + if (getDialect() == Dialect.POSTGRESQL) { + queryBuilder.append("\""); + } viewBuilder.query(queryBuilder.toString()); } @@ -195,18 +249,26 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { Random rnd = getRandom(); int numPks = Math.min(1 + rnd.nextInt(getMaxPkComponents()), MAX_PKS - pkSize); for (int i = 0; i < numPks; i++) { - Column pkColumn = generateColumn(PK_TYPES, -1); + Column pkColumn = + generateColumn( + (getDialect() == Dialect.GOOGLE_STANDARD_SQL) ? PK_TYPES : PG_PK_TYPES, -1); tableBuilder.addColumn(pkColumn); IndexColumn.Order order = rnd.nextBoolean() ? IndexColumn.Order.ASC : IndexColumn.Order.DESC; - IndexColumn pk = IndexColumn.create(pkColumn.name(), order); + if (getDialect() == Dialect.POSTGRESQL) { + order = IndexColumn.Order.ASC; + } + IndexColumn pk = IndexColumn.create(pkColumn.name(), order, getDialect()); tableBuilder.primaryKey().set(pk).end(); } int numColumns = rnd.nextInt(getMaxColumns()); for (int i = 0; i < numColumns; i++) { - Column column = generateColumn(COLUMN_TYPES, getArrayChance()); + Column column = + generateColumn( + (getDialect() == Dialect.GOOGLE_STANDARD_SQL) ? COLUMN_TYPES : PG_COLUMN_TYPES, + getArrayChance()); tableBuilder.addColumn(column); } @@ -215,13 +277,17 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { if (getEnableGeneratedColumns()) { // Add a generated column Column depColumn = table.columns().get(rnd.nextInt(table.columns().size())); + String expr = depColumn.name(); + if (getDialect() == Dialect.POSTGRESQL) { + expr = "\"" + expr + "\""; + } Column generatedColumn = - Column.builder() + Column.builder(getDialect()) .name("generated") .type(depColumn.type()) .max() .notNull(depColumn.notNull()) - .generatedAs(depColumn.name()) + .generatedAs(expr) .stored() .autoBuild(); tableBuilder.addColumn(generatedColumn); @@ -231,8 +297,10 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { int numIndexes = rnd.nextInt(getMaxIndex()); ImmutableList.Builder indexes = ImmutableList.builder(); for (int i = 0; i < numIndexes; i++) { - Index.Builder index = Index.builder().name(generateIdentifier(getMaxIdLength())).table(name); + Index.Builder index = + Index.builder(getDialect()).name(generateIdentifier(getMaxIdLength())).table(name); IndexColumn.IndexColumnsBuilder columns = index.columns(); + ImmutableList.Builder filters = ImmutableList.builder(); boolean interleaved = rnd.nextBoolean(); Set pks = Sets.newHashSet(); // Do not interleave indexes at the last table level. @@ -244,6 +312,9 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { for (IndexColumn pk : table.primaryKeys()) { if (interleaved) { columns.set(pk); + if (rnd.nextBoolean()) { + filters.add("\"" + pk.name() + "\" IS NOT NULL"); + } } pks.add(pk.name()); } @@ -260,34 +331,48 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { if (interleaved && pks.contains(columnName)) { continue; } - if (cm.type().getCode() == Type.Code.ARRAY) { + if (cm.type().getCode() == Type.Code.ARRAY || cm.type().getCode() == Type.Code.PG_ARRAY) { + continue; + } + // Skip the types that may generate NaN value, as NaN cannot be used as a key + if (cm.type().getCode() == Type.Code.FLOAT64 + || cm.type().getCode() == Type.Code.PG_FLOAT8 + || cm.type().getCode() == Type.Code.PG_NUMERIC) { continue; } int val = rnd.nextInt(4); switch (val) { - case 1: - columns.asc(columnName); + case 0: + columns.create().name(columnName).asc(); if (!pks.contains(columnName)) { indexColumns++; } break; - case 2: - columns.desc(columnName); + case 1: + columns.create().name(columnName).desc(); if (!pks.contains(columnName)) { indexColumns++; } break; - case 3: + case 2: if (!pks.contains(columnName)) { - columns.storing(columnName); + columns.create().name(columnName).storing(); } break; default: // skip this column } + // skip the primary key column if it is randomed to storing + if (val < 2 || (val < 3 && !pks.contains(columnName))) { + columns.endIndexColumn(); + if (rnd.nextBoolean()) { + filters.add("\"" + columnName + "\" IS NOT NULL"); + } + } } columns.end(); - // index.nullFiltered(rnd.nextBoolean()); + index.nullFiltered(rnd.nextBoolean()); + index.filter(String.join(" AND ", filters.build())); // index.unique(rnd.nextBoolean()); if (indexColumns > 0) { indexes.add(index.build().prettyPrint()); @@ -301,7 +386,7 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { ImmutableList.Builder foreignKeys = ImmutableList.builder(); for (int i = 0; i < numForeignKeys; i++) { ForeignKey.Builder foreignKeyBuilder = - ForeignKey.builder() + ForeignKey.builder(getDialect()) .name(generateIdentifier(getMaxIdLength())) .table(name) .referencedTable(parent.name()); @@ -323,12 +408,22 @@ private void generateTable(Ddl.Builder builder, Table parent, int level) { ImmutableList columns = table.columns(); int colIndex = rnd.nextInt(columns.size()); Column column = columns.get(colIndex); - if (!CHECK_CONSTRAINT_TYPES.contains(column.type())) { + if (getDialect() == Dialect.GOOGLE_STANDARD_SQL + && !CHECK_CONSTRAINT_TYPES.contains(column.type().getCode())) { + continue; + } + if (getDialect() == Dialect.POSTGRESQL + && !PG_CHECK_CONSTRAINT_TYPES.contains(column.type().getCode())) { continue; } // An expression that won't be trivially optimized away by query optimizer. + String expr = "TO_HEX(SHA1(CAST(" + column.name() + " AS STRING))) <= '~'"; String checkName = generateIdentifier(getMaxIdLength()); + if (getDialect() == Dialect.POSTGRESQL) { + expr = "LENGTH(CAST(\"" + column.name() + "\" AS VARCHAR)) > '-1'::bigint"; + checkName = "\"" + checkName + "\""; + } checkConstraints.add("CONSTRAINT " + checkName + " CHECK(" + expr + ")"); tableBuilder.checkConstraints(checkConstraints.build()); break; @@ -353,7 +448,12 @@ private Column generateColumn(Type.Code[] codes, int arrayPercentage) { Type type = generateType(codes, arrayPercentage); int size = -1; boolean nullable = getRandom().nextBoolean(); - return Column.builder().name(name).type(type).size(size).notNull(nullable).autoBuild(); + return Column.builder(getDialect()) + .name(name) + .type(type) + .size(size) + .notNull(nullable) + .autoBuild(); } private String generateIdentifier(int length) { @@ -371,7 +471,13 @@ private String generateIdentifier(int length) { private Type generateType(Type.Code[] codes, int arrayPercentage) { boolean isArray = getRandom().nextInt(100) <= arrayPercentage; Type.Code code = randomCode(codes); - return isArray ? Type.array(typeOf(code)) : typeOf(code); + if (isArray) { + if (getDialect() == Dialect.POSTGRESQL) { + return Type.pgArray(typeOf(code)); + } + return Type.array(typeOf(code)); + } + return typeOf(code); } private Table selectRandomTable(Ddl.Builder builder) { @@ -403,6 +509,24 @@ private Type typeOf(Type.Code code) { return Type.date(); case INT64: return Type.int64(); + case PG_BOOL: + return Type.pgBool(); + case PG_INT8: + return Type.pgInt8(); + case PG_FLOAT8: + return Type.pgFloat8(); + case PG_TEXT: + return Type.pgText(); + case PG_VARCHAR: + return Type.pgVarchar(); + case PG_BYTEA: + return Type.pgBytea(); + case PG_TIMESTAMPTZ: + return Type.pgTimestamptz(); + case PG_NUMERIC: + return Type.pgNumeric(); + case PG_DATE: + return Type.pgDate(); } throw new IllegalArgumentException("Arrays and Structs are not supported"); } From 93dec07204c0c4a34f1f0743037cb50101426f4a Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 18:48:18 -0700 Subject: [PATCH 133/145] Update on recordConverters for Cloud Spanner to support PG interface PiperOrigin-RevId: 447879122 --- .../teleport/spanner/AvroRecordConverter.java | 61 ++++ .../spanner/BuildReadFromTableOperations.java | 86 +++-- .../spanner/SpannerRecordConverter.java | 145 ++++++-- .../spanner/AvroRecordConverterTest.java | 57 +++ .../spanner/SpannerRecordConverterTest.java | 335 ++++++++++++++++++ 5 files changed, 624 insertions(+), 60 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java b/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java index a5abfe4245..167b04de06 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java @@ -76,27 +76,35 @@ public Mutation apply(GenericRecord record) { switch (column.type().getCode()) { case BOOL: + case PG_BOOL: builder.set(column.name()).to(readBool(record, avroType, fieldName).orElse(null)); break; case INT64: + case PG_INT8: builder.set(column.name()).to(readInt64(record, avroType, fieldName).orElse(null)); break; case FLOAT64: + case PG_FLOAT8: builder.set(column.name()).to(readFloat64(record, avroType, fieldName).orElse(null)); break; case STRING: + case PG_VARCHAR: + case PG_TEXT: case JSON: builder.set(column.name()).to(readString(record, avroType, fieldName).orElse(null)); break; case BYTES: + case PG_BYTEA: builder.set(column.name()).to(readBytes(record, avroType, fieldName).orElse(null)); break; case TIMESTAMP: + case PG_TIMESTAMPTZ: builder .set(column.name()) .to(readTimestamp(record, avroType, logicalType, fieldName).orElse(null)); break; case DATE: + case PG_DATE: builder .set(column.name()) .to(readDate(record, avroType, logicalType, fieldName).orElse(null)); @@ -104,7 +112,11 @@ public Mutation apply(GenericRecord record) { case NUMERIC: builder.set(column.name()).to(readNumeric(record, avroType, fieldName).orElse(null)); break; + case PG_NUMERIC: + builder.set(column.name()).to(readPgNumeric(record, avroType, fieldName).orElse(null)); + break; case ARRAY: + case PG_ARRAY: { Schema arraySchema = avroFieldSchema.getElementType(); if (arraySchema.getType() == Schema.Type.UNION) { @@ -117,32 +129,39 @@ public Mutation apply(GenericRecord record) { Schema.Type arrayType = arraySchema.getType(); switch (column.type().getArrayElementType().getCode()) { case BOOL: + case PG_BOOL: builder .set(column.name()) .toBoolArray(readBoolArray(record, arrayType, fieldName).orElse(null)); break; case INT64: + case PG_INT8: builder .set(column.name()) .toInt64Array(readInt64Array(record, arrayType, fieldName).orElse(null)); break; case FLOAT64: + case PG_FLOAT8: builder .set(column.name()) .toFloat64Array(readFloat64Array(record, arrayType, fieldName).orElse(null)); break; case STRING: + case PG_VARCHAR: + case PG_TEXT: case JSON: builder .set(column.name()) .toStringArray(readStringArray(record, arrayType, fieldName).orElse(null)); break; case BYTES: + case PG_BYTEA: builder .set(column.name()) .toBytesArray(readBytesArray(record, arrayType, fieldName).orElse(null)); break; case TIMESTAMP: + case PG_TIMESTAMPTZ: builder .set(column.name()) .toTimestampArray( @@ -150,6 +169,7 @@ public Mutation apply(GenericRecord record) { .orElse(null)); break; case DATE: + case PG_DATE: builder .set(column.name()) .toDateArray(readDateArray(record, arrayType, fieldName).orElse(null)); @@ -159,6 +179,12 @@ public Mutation apply(GenericRecord record) { .set(column.name()) .toStringArray(readNumericArray(record, arrayType, fieldName).orElse(null)); break; + case PG_NUMERIC: + builder + .set(column.name()) + .toPgNumericArray( + readPgNumericArray(record, arrayType, fieldName).orElse(null)); + break; default: throw new IllegalArgumentException( String.format( @@ -252,6 +278,29 @@ static Optional> readNumericArray( } } + @VisibleForTesting + @SuppressWarnings("unchecked") + static Optional> readPgNumericArray( + GenericRecord record, Schema.Type avroType, String fieldName) { + Object fieldValue = record.get(fieldName); + if (fieldValue == null) { + return Optional.empty(); + } + switch (avroType) { + case BYTES: + List values = (List) record.get(fieldName); + if (values == null) { + return Optional.empty(); + } + return Optional.of( + values.stream() + .map(x -> x == null ? null : NumericUtils.pgBytesToString(x.array())) + .collect(Collectors.toList())); + default: + throw new IllegalArgumentException("Cannot interpret " + avroType + " as NUMERIC"); + } + } + @VisibleForTesting @SuppressWarnings("unchecked") static Optional> readTimestampArray( @@ -462,6 +511,18 @@ private Optional readNumeric( } } + static Optional readPgNumeric( + GenericRecord record, Schema.Type avroType, String fieldName) { + switch (avroType) { + case BYTES: + return Optional.ofNullable((ByteBuffer) record.get(fieldName)) + .map(ByteBuffer::array) + .map(NumericUtils::pgBytesToString); + default: + throw new IllegalArgumentException("Cannot interpret " + avroType + " as NUMERIC"); + } + } + private Optional readTimestamp( GenericRecord record, Schema.Type avroType, LogicalType logicalType, String fieldName) { switch (avroType) { diff --git a/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java b/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java index 782b6a2f92..f20b823eab 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java +++ b/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java @@ -80,13 +80,30 @@ public void processElement(ProcessContext c) { // Also have to export table name to be able to identify which row belongs to // which table. - ReadOperation read = - ReadOperation.create() - .withQuery( - String.format( - "SELECT \"%s\" AS _spanner_table, %s FROM `%s` AS t", - table.name(), columnsListAsString, table.name())) - .withPartitionOptions(partitionOptions); + ReadOperation read; + switch (ddl.dialect()) { + case GOOGLE_STANDARD_SQL: + read = + ReadOperation.create() + .withQuery( + String.format( + "SELECT \"%s\" AS _spanner_table, %s FROM `%s` AS t", + table.name(), columnsListAsString, table.name())) + .withPartitionOptions(partitionOptions); + break; + case POSTGRESQL: + read = + ReadOperation.create() + .withQuery( + String.format( + "SELECT '%s' AS _spanner_table, %s FROM \"%s\" AS t", + table.name(), columnsListAsString, table.name())) + .withPartitionOptions(partitionOptions); + break; + default: + throw new IllegalArgumentException( + String.format("Unrecognized dialect: %s", ddl.dialect())); + } c.output(read); } } @@ -94,28 +111,39 @@ public void processElement(ProcessContext c) { } private String createColumnExpression(Column col) { - if (col.typeString().equals("NUMERIC")) { - return "CAST(" + "t.`" + col.name() + "`" + " AS STRING) AS " + col.name(); + switch (col.dialect()) { + case GOOGLE_STANDARD_SQL: + if (col.typeString().equals("NUMERIC")) { + return "CAST(" + "t.`" + col.name() + "`" + " AS STRING) AS " + col.name(); + } + if (col.typeString().equals("JSON")) { + return "TO_JSON_STRING(" + "t.`" + col.name() + "`" + ") AS " + col.name(); + } + if (col.typeString().equals("ARRAY")) { + return "(SELECT ARRAY_AGG(CAST(num AS STRING)) FROM UNNEST(" + + "t.`" + + col.name() + + "`" + + ") AS num) AS " + + col.name(); + } + if (col.typeString().equals("ARRAY")) { + return "(SELECT ARRAY_AGG(TO_JSON_STRING(element)) FROM UNNEST(" + + "t.`" + + col.name() + + "`" + + ") AS element) AS " + + col.name(); + } + return "t.`" + col.name() + "`"; + case POSTGRESQL: + if (col.typeString().equals("numeric")) { + return "CAST(" + "t.\"" + col.name() + "\"" + " AS VARCHAR) AS " + col.name(); + } + return "t.\"" + col.name() + "\""; + default: + throw new IllegalArgumentException( + String.format("Unrecognized dialect: %s", col.dialect())); } - if (col.typeString().equals("JSON")) { - return "TO_JSON_STRING(" + "t.`" + col.name() + "`" + ") AS " + col.name(); - } - if (col.typeString().equals("ARRAY")) { - return "(SELECT ARRAY_AGG(CAST(num AS STRING)) FROM UNNEST(" - + "t.`" - + col.name() - + "`" - + ") AS num) AS " - + col.name(); - } - if (col.typeString().equals("ARRAY")) { - return "(SELECT ARRAY_AGG(TO_JSON_STRING(element)) FROM UNNEST(" - + "t.`" - + col.name() - + "`" - + ") AS element) AS " - + col.name(); - } - return "t.`" + col.name() + "`"; } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java b/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java index f7e2eed663..45f98b254f 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java @@ -18,6 +18,7 @@ import com.google.cloud.Timestamp; import com.google.cloud.spanner.Struct; import com.google.cloud.teleport.spanner.common.NumericUtils; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.base.Strings; import java.nio.ByteBuffer; import java.util.List; @@ -32,10 +33,21 @@ public class SpannerRecordConverter { private static final Pattern STRING_PATTERN = Pattern.compile("STRING\\((?:MAX|[0-9]+)\\)"); private static final Pattern ARRAY_PATTERN = Pattern.compile("ARRAY"); + private static final Pattern VARCHAR_PATTERN = + Pattern.compile("character varying(\\([0-9]+\\))?"); + private static final Pattern VARCHAR_ARRAY_PATTERN = + Pattern.compile("character varying(\\([0-9]+\\))?\\[\\]"); private final Schema schema; + private final Dialect dialect; + + public SpannerRecordConverter(Schema schema, Dialect dialect) { + this.schema = schema; + this.dialect = dialect; + } public SpannerRecordConverter(Schema schema) { this.schema = schema; + this.dialect = Dialect.GOOGLE_STANDARD_SQL; } public GenericRecord convert(Struct row) { @@ -69,7 +81,9 @@ public GenericRecord convert(Struct row) { builder.set(field, nullValue ? null : row.getBoolean(fieldName)); break; case LONG: - if (spannerType.equals("TIMESTAMP")) { + if ((dialect == Dialect.GOOGLE_STANDARD_SQL && spannerType.equals("TIMESTAMP")) + || (dialect == Dialect.POSTGRESQL + && spannerType.equals("timestamp with time zone"))) { long microSeconds = 0L; if (!nullValue) { Timestamp ts = row.getTimestamp(fieldName); @@ -86,7 +100,7 @@ public GenericRecord convert(Struct row) { builder.set(field, nullValue ? null : row.getDouble(fieldName)); break; case BYTES: - if (spannerType.equals("NUMERIC")) { + if (dialect == Dialect.GOOGLE_STANDARD_SQL && spannerType.equals("NUMERIC")) { // TODO: uses row.getNumeric() once teleport uses new spanner library. builder.set( field, @@ -95,16 +109,34 @@ public GenericRecord convert(Struct row) { : ByteBuffer.wrap(NumericUtils.stringToBytes(row.getString(fieldName)))); break; } + if (dialect == Dialect.POSTGRESQL && spannerType.equals("numeric")) { + builder.set( + field, + nullValue + ? null + : ByteBuffer.wrap(NumericUtils.pgStringToBytes(row.getString(fieldName)))); + break; + } builder.set( field, nullValue ? null : ByteBuffer.wrap(row.getBytes(fieldName).toByteArray())); break; case STRING: - if (STRING_PATTERN.matcher(spannerType).matches() || spannerType.equals("JSON")) { - builder.set(field, nullValue ? null : row.getString(fieldName)); - } else if (spannerType.equals("TIMESTAMP")) { - builder.set(field, nullValue ? null : row.getTimestamp(fieldName).toString()); - } else if (spannerType.equals("DATE")) { - builder.set(field, nullValue ? null : row.getDate(fieldName).toString()); + if (dialect == Dialect.GOOGLE_STANDARD_SQL) { + if (STRING_PATTERN.matcher(spannerType).matches() || spannerType.equals("JSON")) { + builder.set(field, nullValue ? null : row.getString(fieldName)); + } else if (spannerType.equals("TIMESTAMP")) { + builder.set(field, nullValue ? null : row.getTimestamp(fieldName).toString()); + } else if (spannerType.equals("DATE")) { + builder.set(field, nullValue ? null : row.getDate(fieldName).toString()); + } + } else if (dialect == Dialect.POSTGRESQL) { + if (VARCHAR_PATTERN.matcher(spannerType).matches() || spannerType.equals("text")) { + builder.set(field, nullValue ? null : row.getString(fieldName)); + } else if (spannerType.equals("timestamp with time zone")) { + builder.set(field, nullValue ? null : row.getTimestamp(fieldName).toString()); + } else if (spannerType.equals("date")) { + builder.set(field, nullValue ? null : row.getDate(fieldName).toString()); + } } break; case ARRAY: @@ -126,7 +158,10 @@ public GenericRecord convert(Struct row) { builder.set(field, nullValue ? null : row.getBooleanList(fieldName)); break; case LONG: - if (spannerType.equals("ARRAY")) { + if ((dialect == Dialect.GOOGLE_STANDARD_SQL + && spannerType.equals("ARRAY")) + || (dialect == Dialect.POSTGRESQL + && spannerType.equals("timestamp with time zone[]"))) { List values = row.getTimestampList(fieldName).stream() .map( @@ -148,7 +183,8 @@ public GenericRecord convert(Struct row) { } case BYTES: { - if (spannerType.equals("ARRAY")) { + if (dialect == Dialect.GOOGLE_STANDARD_SQL + && spannerType.equals("ARRAY")) { if (nullValue) { builder.set(field, null); break; @@ -165,6 +201,23 @@ public GenericRecord convert(Struct row) { builder.set(field, numericValues); break; } + if (dialect == Dialect.POSTGRESQL && spannerType.equals("numeric[]")) { + if (nullValue) { + builder.set(field, null); + break; + } + List numericValues = null; + numericValues = + row.getStringList(fieldName).stream() + .map( + numeric -> + numeric == null + ? null + : ByteBuffer.wrap(NumericUtils.pgStringToBytes(numeric))) + .collect(Collectors.toList()); + builder.set(field, numericValues); + break; + } List value = null; if (!nullValue) { value = @@ -179,28 +232,24 @@ public GenericRecord convert(Struct row) { } case STRING: { - if (ARRAY_PATTERN.matcher(spannerType).matches() - || spannerType.equals("ARRAY")) { - builder.set(field, nullValue ? null : row.getStringList(fieldName)); - } else if (spannerType.equals("ARRAY")) { - if (nullValue) { - builder.set(field, null); - } else { - List values = - row.getTimestampList(fieldName).stream() - .map(timestamp -> timestamp == null ? null : timestamp.toString()) - .collect(Collectors.toList()); - builder.set(field, values); + if (dialect == Dialect.GOOGLE_STANDARD_SQL) { + if (ARRAY_PATTERN.matcher(spannerType).matches() + || spannerType.equals("ARRAY")) { + builder.set(field, nullValue ? null : row.getStringList(fieldName)); + } else if (spannerType.equals("ARRAY")) { + setTimestampArray(row, builder, field, fieldName, nullValue); + } else if (spannerType.equals("ARRAY")) { + setDateArray(row, builder, field, fieldName, nullValue); } - } else if (spannerType.equals("ARRAY")) { - if (nullValue) { - builder.set(field, null); - } else { - List values = - row.getDateList(fieldName).stream() - .map(date -> date == null ? null : date.toString()) - .collect(Collectors.toList()); - builder.set(field, values); + } + if (dialect == Dialect.POSTGRESQL) { + if (VARCHAR_ARRAY_PATTERN.matcher(spannerType).matches() + || spannerType.equals("text[]")) { + builder.set(field, nullValue ? null : row.getStringList(fieldName)); + } else if (spannerType.equals("timestamp with time zone[]")) { + setTimestampArray(row, builder, field, fieldName, nullValue); + } else if (spannerType.equals("date[]")) { + setDateArray(row, builder, field, fieldName, nullValue); } } break; @@ -220,4 +269,38 @@ public GenericRecord convert(Struct row) { } return builder.build(); } + + private static void setTimestampArray( + Struct row, + GenericRecordBuilder builder, + Schema.Field field, + String fieldName, + boolean nullValue) { + if (nullValue) { + builder.set(field, null); + } else { + List values = + row.getTimestampList(fieldName).stream() + .map(timestamp -> timestamp == null ? null : timestamp.toString()) + .collect(Collectors.toList()); + builder.set(field, values); + } + } + + private static void setDateArray( + Struct row, + GenericRecordBuilder builder, + Schema.Field field, + String fieldName, + boolean nullValue) { + if (nullValue) { + builder.set(field, null); + } else { + List values = + row.getDateList(fieldName).stream() + .map(date -> date == null ? null : date.toString()) + .collect(Collectors.toList()); + builder.set(field, values); + } + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/AvroRecordConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/AvroRecordConverterTest.java index e8f69d83b0..dea1464ab3 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/AvroRecordConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/AvroRecordConverterTest.java @@ -330,4 +330,61 @@ public void timestampArray() { avroRecord, LONG, LogicalTypes.timestampMillis(), colName); assertArrayEquals(expectedTimestamps, result.get().toArray()); } + + @Test + public void pgNumericArray() { + String colName = "arrayOfPgNumeric"; + Schema schema = + SchemaBuilder.record("record") + .fields() + .requiredBytes("id") + .name(colName) + .type() + .optional() + .array() + .items() + .bytesType() + .endRecord(); + + // Null field + GenericRecord avroRecord = new GenericRecordBuilder(schema).set("id", 0).build(); + Optional> result = + AvroRecordConverter.readPgNumericArray(avroRecord, BYTES, colName); + assertFalse(result.isPresent()); + + StringBuilder maxPgNumeric = new StringBuilder(); + StringBuilder minPgNumeric = new StringBuilder("-"); + for (int i = 0; i < NumericUtils.PG_MAX_PRECISION - NumericUtils.PG_MAX_SCALE; i++) { + maxPgNumeric.append("9"); + minPgNumeric.append("9"); + } + maxPgNumeric.append("."); + minPgNumeric.append("."); + for (int i = 0; i < NumericUtils.PG_MAX_SCALE; i++) { + maxPgNumeric.append("9"); + minPgNumeric.append("9"); + } + + String[] readablePgNumericValues = { + "-123.456000000", + "0.000000000", + "2387653.235320000", + null, + "0.000000020", + "-99999999999999999999999999999.999999999", + "NaN", + "100000000000000000000001.000001000", + maxPgNumeric.toString(), + minPgNumeric.toString() + }; + + List avroPgNumericValues = + Stream.of(readablePgNumericValues) + .map(x -> x == null ? null : ByteBuffer.wrap(NumericUtils.pgStringToBytes(x))) + .collect(Collectors.toList()); + avroRecord = + new GenericRecordBuilder(schema).set("id", 0).set(colName, avroPgNumericValues).build(); + result = AvroRecordConverter.readPgNumericArray(avroRecord, BYTES, colName); + assertArrayEquals(readablePgNumericValues, result.get().toArray()); + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java index 2e9850c533..fb813826dc 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java @@ -26,6 +26,7 @@ import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.collect.Lists; import java.nio.ByteBuffer; import java.util.Arrays; @@ -388,4 +389,338 @@ public void json() { Arrays.asList( null, "[1,null,true,2.2523,\"hello\"]", null, "{\"a\":{\"a\":2.5},\"b\":null}"))); } + + @Test + public void pgSimple() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("email") + .pgVarchar() + .size(15) + .notNull() + .endColumn() + .column("name") + .pgVarchar() + .max() + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + Schema schema = converter.convert(ddl).iterator().next(); + SpannerRecordConverter recordConverter = new SpannerRecordConverter(schema, Dialect.POSTGRESQL); + Struct struct = + Struct.newBuilder() + .set("id") + .to(1L) + .set("email") + .to("abc@google.com") + .set("name") + .to("John Doe") + .build(); + + GenericRecord avroRecord = recordConverter.convert(struct); + + assertThat(avroRecord.get("id"), equalTo(1L)); + assertThat(avroRecord.get("email"), equalTo("abc@google.com")); + assertThat(avroRecord.get("name"), equalTo("John Doe")); + } + + @Test + public void pgNulls() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("age") + .pgInt8() + .endColumn() + .column("name") + .pgVarchar() + .max() + .endColumn() + .column("bytes") + .pgBytea() + .endColumn() + .column("ts") + .pgTimestamptz() + .endColumn() + .column("date") + .pgDate() + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + Schema schema = converter.convert(ddl).iterator().next(); + SpannerRecordConverter recordConverter = new SpannerRecordConverter(schema, Dialect.POSTGRESQL); + Struct struct = + Struct.newBuilder() + .set("id") + .to(1L) + .set("age") + .to((Long) null) + .set("name") + .to((String) null) + .set("bytes") + .to((ByteArray) null) + .set("ts") + .to((Timestamp) null) + .set("date") + .to((Date) null) + .build(); + + GenericRecord avroRecord = recordConverter.convert(struct); + + assertThat(avroRecord.get("id"), equalTo(1L)); + assertThat(avroRecord.get("age"), is((Long) null)); + assertThat(avroRecord.get("name"), is((String) null)); + assertThat(avroRecord.get("bytes"), is((ByteArray) null)); + assertThat(avroRecord.get("ts"), is((String) null)); + assertThat(avroRecord.get("date"), is((String) null)); + } + + @Test + public void pgDateTimestamptz() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("ts") + .pgTimestamptz() + .endColumn() + .column("date") + .pgDate() + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + Schema schema = converter.convert(ddl).iterator().next(); + SpannerRecordConverter recordConverter = new SpannerRecordConverter(schema, Dialect.POSTGRESQL); + Struct struct = + Struct.newBuilder() + .set("id") + .to(1L) + .set("ts") + .to(Timestamp.ofTimeMicroseconds(10)) + .set("date") + .to(Date.fromYearMonthDay(2018, 2, 2)) + .build(); + + GenericRecord avroRecord = recordConverter.convert(struct); + + assertThat(avroRecord.get("id"), equalTo(1L)); + assertThat(avroRecord.get("ts"), equalTo("1970-01-01T00:00:00.000010000Z")); + assertThat(avroRecord.get("date"), equalTo("2018-02-02")); + } + + @Test + public void pgTimestamptzLogical() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("ts1") + .pgTimestamptz() + .endColumn() + .column("ts2") + .pgTimestamptz() + .endColumn() + .column("ts3") + .pgTimestamptz() + .endColumn() + .column("ts4") + .pgTimestamptz() + .endColumn() + .column("ts5") + .pgTimestamptz() + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + Schema schema = logicalTypeConverter.convert(ddl).iterator().next(); + SpannerRecordConverter recordConverter = new SpannerRecordConverter(schema, Dialect.POSTGRESQL); + Struct struct = + Struct.newBuilder() + .set("id") + .to(1L) + .set("ts1") + .to(Timestamp.ofTimeMicroseconds(10)) + .set("ts2") + .to(Timestamp.ofTimeSecondsAndNanos(10000, 100000)) + .set("ts3") + .to(Timestamp.parseTimestamp("1970-01-01T00:00:00Z")) + .set("ts4") + .to(Timestamp.MIN_VALUE) + .set("ts5") + .to(Timestamp.MAX_VALUE) + .build(); + + GenericRecord avroRecord = recordConverter.convert(struct); + + assertThat(avroRecord.get("id"), equalTo(1L)); + assertThat(avroRecord.get("ts1"), equalTo(10L)); + assertThat(avroRecord.get("ts2"), equalTo(10000000100L)); + assertThat(avroRecord.get("ts3"), equalTo(0L)); + assertThat(avroRecord.get("ts4"), equalTo(-62135596800000000L)); + assertThat(avroRecord.get("ts5"), equalTo(253402300799999999L)); + } + + @Test + public void pgNumerics() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("numerictable") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("numeric1") + .type(Type.pgNumeric()) + .endColumn() + .column("numeric2") + .type(Type.pgNumeric()) + .endColumn() + .column("numeric3") + .type(Type.pgNumeric()) + .endColumn() + .column("numeric_arr") + .type(Type.pgArray(Type.pgNumeric())) + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + Schema schema = converter.convert(ddl).iterator().next(); + SpannerRecordConverter recordConverter = new SpannerRecordConverter(schema, Dialect.POSTGRESQL); + + StringBuilder maxPgNumeric = new StringBuilder(); + StringBuilder minPgNumeric = new StringBuilder("-"); + for (int i = 0; i < NumericUtils.PG_MAX_PRECISION - NumericUtils.PG_MAX_SCALE; i++) { + maxPgNumeric.append("9"); + minPgNumeric.append("9"); + } + maxPgNumeric.append("."); + minPgNumeric.append("."); + for (int i = 0; i < NumericUtils.PG_MAX_SCALE; i++) { + maxPgNumeric.append("9"); + minPgNumeric.append("9"); + } + + String[] pgNumericArrValues = { + null, "NaN", null, maxPgNumeric.toString(), minPgNumeric.toString() + }; + + Struct struct = + Struct.newBuilder() + .set("id") + .to(1L) + .set("numeric1") + .to("-9305028.140032") + .set("numeric2") + .to("-25398514232141142.012479") + .set("numeric3") + .to("1999999999.1246") + .set("numeric_arr") + .toStringArray(Lists.newArrayList(pgNumericArrValues)) + .build(); + + GenericRecord avroRecord = recordConverter.convert(struct); + List expectedPgNumericArr = + Stream.of(pgNumericArrValues) + .map(x -> x == null ? null : ByteBuffer.wrap(NumericUtils.pgStringToBytes(x))) + .collect(Collectors.toList()); + + assertThat(avroRecord.get("id"), equalTo(1L)); + assertThat( + avroRecord.get("numeric1"), + equalTo(ByteBuffer.wrap(NumericUtils.pgStringToBytes("-9305028.140032")))); + assertThat( + avroRecord.get("numeric2"), + equalTo(ByteBuffer.wrap(NumericUtils.pgStringToBytes("-25398514232141142.012479")))); + assertThat( + avroRecord.get("numeric3"), + equalTo(ByteBuffer.wrap(NumericUtils.pgStringToBytes("1999999999.1246")))); + assertThat(avroRecord.get("numeric_arr"), equalTo(expectedPgNumericArr)); + } + + @Test + public void pgArrays() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("users") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("ints") + .type(Type.pgArray(Type.pgInt8())) + .endColumn() + .column("varchars") + .type(Type.pgArray(Type.pgVarchar())) + .max() + .endColumn() + .column("texts") + .type(Type.pgArray(Type.pgText())) + .endColumn() + .column("ts") + .type(Type.pgArray(Type.pgTimestamptz())) + .endColumn() + .column("date") + .type(Type.pgArray(Type.pgDate())) + .endColumn() + .primaryKey() + .asc("id") + .end() + .endTable() + .build(); + Schema schema = converter.convert(ddl).iterator().next(); + SpannerRecordConverter recordConverter = new SpannerRecordConverter(schema, Dialect.POSTGRESQL); + Struct struct = + Struct.newBuilder() + .set("id") + .to(1L) + .set("ints") + .toInt64Array(Lists.newArrayList(1L, null, 2L)) + .set("varchars") + .toStringArray(Lists.newArrayList(null, null, "one")) + .set("texts") + .toStringArray(Lists.newArrayList(null, null, "two")) + .set("ts") + .toTimestampArray(Lists.newArrayList(null, null, Timestamp.ofTimeMicroseconds(10L))) + .set("date") + .toDateArray(Lists.newArrayList(null, null, Date.fromYearMonthDay(2018, 2, 2))) + .build(); + + GenericRecord avroRecord = recordConverter.convert(struct); + + assertThat(avroRecord.get("id"), equalTo(1L)); + assertThat(avroRecord.get("ints"), equalTo(Arrays.asList(1L, null, 2L))); + assertThat(avroRecord.get("varchars"), equalTo(Arrays.asList(null, null, "one"))); + assertThat(avroRecord.get("texts"), equalTo(Arrays.asList(null, null, "two"))); + assertThat(avroRecord.get("date"), equalTo(Arrays.asList(null, null, "2018-02-02"))); + assertThat( + avroRecord.get("ts"), equalTo(Arrays.asList(null, null, "1970-01-01T00:00:00.000010000Z"))); + } } From 8a6a15bd1f8fa7cb8cffca2def7b6534a64fbb41 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Tue, 10 May 2022 23:17:40 -0700 Subject: [PATCH 134/145] Update text templates to support Spanner's PG interface PiperOrigin-RevId: 447914126 --- .../teleport/spanner/TextImportTransform.java | 47 +++-- .../teleport/spanner/TextRowToMutation.java | 9 + .../teleport/templates/SpannerToText.java | 2 +- .../templates/common/SpannerConverters.java | 99 +++++++++-- .../spanner/TextImportTransformTest.java | 116 +++++++++++- .../spanner/TextRowToMutationTest.java | 168 ++++++++++++++++++ .../templates/SpannerServerResource.java | 21 ++- .../teleport/templates/SpannerToTextTest.java | 97 ++++++++-- .../common/SpannerConverterTest.java | 58 +++++- 9 files changed, 569 insertions(+), 48 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java index 7b29fc7d86..615907f036 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java @@ -21,6 +21,7 @@ import com.google.cloud.teleport.spanner.common.Type.Code; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashMultimap; @@ -405,26 +406,47 @@ public void processElement(ProcessContext c) { .withSideInputs(ddlView)); } - public static Code parseSpannerDataType(String columnType) { - if (STRING_PATTERN.matcher(columnType).matches()) { + public static Code parseSpannerDataType(String columnType, Dialect dialect) { + if (STRING_PATTERN.matcher(columnType).matches() && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.STRING; - } else if (columnType.equalsIgnoreCase("INT64")) { + } else if (columnType.equalsIgnoreCase("INT64") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.INT64; - } else if (columnType.equalsIgnoreCase("FLOAT64")) { + } else if (columnType.equalsIgnoreCase("FLOAT64") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.FLOAT64; - } else if (columnType.equalsIgnoreCase("BOOL")) { + } else if (columnType.equalsIgnoreCase("BOOL") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.BOOL; - } else if (columnType.equalsIgnoreCase("DATE")) { + } else if (columnType.equalsIgnoreCase("DATE") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.DATE; - } else if (columnType.equalsIgnoreCase("TIMESTAMP")) { + } else if (columnType.equalsIgnoreCase("TIMESTAMP") + && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.TIMESTAMP; - } else if (columnType.equalsIgnoreCase("BYTES")) { + } else if (columnType.equalsIgnoreCase("BYTES") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.BYTES; - } else if (columnType.equalsIgnoreCase("NUMERIC")) { + } else if (columnType.equalsIgnoreCase("NUMERIC") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.NUMERIC; - } else if (columnType.equalsIgnoreCase("JSON")) { + } else if (columnType.equalsIgnoreCase("JSON") && dialect == Dialect.GOOGLE_STANDARD_SQL) { return Code.JSON; - } else { + } else if (columnType.equalsIgnoreCase("bigint") && dialect == Dialect.POSTGRESQL) { + return Code.PG_INT8; + } else if (columnType.equalsIgnoreCase("double precision") && dialect == Dialect.POSTGRESQL) { + return Code.PG_FLOAT8; + } else if (columnType.equalsIgnoreCase("boolean") && dialect == Dialect.POSTGRESQL) { + return Code.PG_BOOL; + } else if (columnType.equalsIgnoreCase("timestamp with time zone") + && dialect == Dialect.POSTGRESQL) { + return Code.PG_TIMESTAMPTZ; + } else if (columnType.equalsIgnoreCase("bytea") && dialect == Dialect.POSTGRESQL) { + return Code.PG_BYTEA; + } else if (columnType.equalsIgnoreCase("numeric") && dialect == Dialect.POSTGRESQL) { + return Code.PG_NUMERIC; + } else if (columnType.toLowerCase().startsWith("character varying") + && dialect == Dialect.POSTGRESQL) { + return Code.PG_VARCHAR; + } else if (columnType.equalsIgnoreCase("text") && dialect == Dialect.POSTGRESQL) { + return Code.PG_TEXT; + } else if (columnType.equalsIgnoreCase("date") && dialect == Dialect.POSTGRESQL) { + return Code.PG_DATE; + }else { throw new IllegalArgumentException( "Unrecognized or unsupported column data type: " + columnType); } @@ -465,7 +487,8 @@ private static void validateManifest(TableManifest tableManifest, Ddl ddl) { + "Generated columns cannot be imported.", manifiestColumn.getColumnName(), table.name())); } - if (parseSpannerDataType(manifiestColumn.getTypeName()) != dbColumn.type().getCode()) { + if (parseSpannerDataType(manifiestColumn.getTypeName(), ddl.dialect()) + != dbColumn.type().getCode()) { throw new RuntimeException( String.format( "Mismatching type: Table %s Column %s [%s from DB and %s from manifest]", diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java b/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java index e6d66ce7e7..8dcf9964cf 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java @@ -170,6 +170,7 @@ protected final Mutation parseRow( // in terms of how input is accepted, and throw exceptions on invalid input. switch (columnType.getCode()) { case BOOL: + case PG_BOOL: if (isNullValue) { columnValue = Value.bool(null); } else { @@ -186,17 +187,22 @@ protected final Mutation parseRow( } break; case INT64: + case PG_INT8: columnValue = isNullValue ? Value.int64(null) : Value.int64(Long.valueOf(cellValue.trim())); break; case FLOAT64: + case PG_FLOAT8: columnValue = isNullValue ? Value.float64(null) : Value.float64(Double.valueOf(cellValue.trim())); break; case STRING: + case PG_VARCHAR: + case PG_TEXT: columnValue = Value.string(cellValue); break; case DATE: + case PG_DATE: if (isNullValue) { columnValue = Value.date(null); } else { @@ -214,6 +220,7 @@ protected final Mutation parseRow( } break; case TIMESTAMP: + case PG_TIMESTAMPTZ: if (isNullValue) { columnValue = Value.timestamp(null); } else { @@ -238,9 +245,11 @@ protected final Mutation parseRow( break; case NUMERIC: case JSON: + case PG_NUMERIC: columnValue = isNullValue ? Value.string(null) : Value.string(cellValue.trim()); break; case BYTES: + case PG_BYTEA: columnValue = isNullValue ? Value.bytes(null) : Value.bytes(ByteArray.fromBase64(cellValue.trim())); break; diff --git a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java index bfafd303fe..582bf35510 100644 --- a/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java +++ b/src/main/java/com/google/cloud/teleport/templates/SpannerToText.java @@ -78,7 +78,7 @@ public class SpannerToText { /** Custom PipelineOptions. */ public interface SpannerToTextOptions extends PipelineOptions, SpannerReadOptions, FilesystemWriteOptions { - + @Description("Temporary Directory to store Csv files.") ValueProvider getCsvTempDirectory(); diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java index 99def36020..636f1aab64 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java @@ -30,6 +30,7 @@ import com.google.cloud.spanner.Type; import com.google.cloud.spanner.Type.Code; import com.google.cloud.spanner.Type.StructField; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -222,6 +223,7 @@ public void processElement(ProcessContext processContext) { // Save schema to GCS so it can be saved along with the exported file. LOG.info("Creating database client for schema read"); + Dialect dialect; LinkedHashMap columns; try { DatabaseClient databaseClient = getDatabaseClient(spannerConfig()); @@ -229,8 +231,10 @@ public void processElement(ProcessContext processContext) { TimestampBound tsbound = getTimestampBound(timestampString); try (ReadOnlyTransaction context = databaseClient.readOnlyTransaction(tsbound)) { + LOG.info("Reading dialect information"); + dialect = getDialect(context); LOG.info("Reading schema information"); - columns = getAllColumns(context, table().get()); + columns = getAllColumns(context, table().get(), dialect); String columnJson = SpannerConverters.GSON.toJson(columns); LOG.info("Saving schema information"); saveSchema(columnJson, textWritePrefix().get() + SCHEMA_SUFFIX); @@ -244,16 +248,37 @@ public void processElement(ProcessContext processContext) { String columnsListAsString = columns.entrySet().stream() - .map(x -> createColumnExpression(x.getKey(), x.getValue())) + .map(x -> createColumnExpression(x.getKey(), x.getValue(), dialect)) .collect(Collectors.joining(",")); - ReadOperation read = - ReadOperation.create() - .withQuery(String.format("SELECT %s FROM `%s`", columnsListAsString, table().get())) - .withPartitionOptions(partitionOptions); + ReadOperation read; + switch (dialect) { + case GOOGLE_STANDARD_SQL: + read = + ReadOperation.create() + .withQuery( + String.format("SELECT %s FROM `%s`", columnsListAsString, table().get())) + .withPartitionOptions(partitionOptions); + break; + case POSTGRESQL: + read = + ReadOperation.create() + .withQuery( + String.format("SELECT %s FROM \"%s\";", columnsListAsString, table().get())) + .withPartitionOptions(partitionOptions); + break; + default: + throw new IllegalArgumentException(String.format("Unrecognized dialect: %s", dialect)); + } processContext.output(read); } - private String createColumnExpression(String columnName, String columnType) { + private String createColumnExpression(String columnName, String columnType, Dialect dialect) { + if (dialect == Dialect.POSTGRESQL) { + if (columnType.equals("numeric")) { + return "CAST(\"" + columnName + "\" AS VARCHAR) AS " + columnName; + } + return "\"" + columnName + "\""; + } if (columnType.equals("NUMERIC")) { return "CAST(`" + columnName + "` AS STRING) AS " + columnName; } @@ -292,18 +317,39 @@ private void saveSchema(String content, String schemaPath) { } /** Function to get all column names from the table. */ - private LinkedHashMap getAllColumns(ReadContext context, String tableName) { + private LinkedHashMap getAllColumns( + ReadContext context, String tableName, Dialect dialect) { LinkedHashMap columns = Maps.newLinkedHashMap(); - ResultSet resultSet = - context.executeQuery( - Statement.newBuilder( - "SELECT COLUMN_NAME, SPANNER_TYPE FROM INFORMATION_SCHEMA.COLUMNS " - + "WHERE TABLE_NAME=@table_name AND TABLE_CATALOG='' AND TABLE_SCHEMA='' " - + "AND IS_GENERATED = 'NEVER' " - + "ORDER BY ORDINAL_POSITION") - .bind("table_name") - .to(tableName) - .build()); + String statement; + ResultSet resultSet; + switch (dialect) { + case GOOGLE_STANDARD_SQL: + statement = + "SELECT COLUMN_NAME, SPANNER_TYPE FROM INFORMATION_SCHEMA.COLUMNS " + + "WHERE TABLE_NAME=@table_name AND TABLE_CATALOG='' AND TABLE_SCHEMA='' " + + "AND IS_GENERATED = 'NEVER' " + + "ORDER BY ORDINAL_POSITION"; + resultSet = + context.executeQuery( + Statement.newBuilder(statement).bind("table_name").to(tableName).build()); + break; + case POSTGRESQL: + statement = + "SELECT COLUMN_NAME, SPANNER_TYPE FROM INFORMATION_SCHEMA.COLUMNS WHERE" + + " TABLE_NAME=$1 AND TABLE_CATALOG=$2 AND TABLE_SCHEMA='public'" + + " AND IS_GENERATED = 'NEVER' ORDER BY ORDINAL_POSITION;"; + resultSet = + context.executeQuery( + Statement.newBuilder(statement) + .bind("p1") + .to(tableName) + .bind("p2") + .to(spannerConfig().getDatabaseId().get()) + .build()); + break; + default: + throw new IllegalArgumentException(String.format("Unrecognized dialect: %s", dialect)); + } LOG.info("Got schema information. Reading columns."); while (resultSet.next()) { Struct currentRow = resultSet.getCurrentRowAsStruct(); @@ -311,6 +357,23 @@ private LinkedHashMap getAllColumns(ReadContext context, String } return columns; } + + private Dialect getDialect(ReadContext context) { + String dialect = ""; + Statement statement = + Statement.of( + "SELECT CASE schema_name WHEN 'information_schema' THEN 'POSTGRESQL' WHEN" + + " 'INFORMATION_SCHEMA' THEN 'GOOGLE_STANDARD_SQL' END dialect FROM" + + " information_schema.schemata WHERE schema_name IN('information_schema'," + + " 'INFORMATION_SCHEMA');"); + ResultSet resultSet = context.executeQuery(statement); + LOG.info("Got dialect information."); + if (resultSet.next()) { + Struct currentRow = resultSet.getCurrentRowAsStruct(); + dialect = currentRow.getString(0); + } + return Dialect.valueOf(dialect); + } } /** diff --git a/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java b/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java index 7021f5da8a..7961888e3a 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java @@ -17,10 +17,12 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import com.google.cloud.teleport.spanner.TextImportTransform.ReadImportManifest; import com.google.cloud.teleport.spanner.TextImportTransform.ResolveDataFiles; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import java.io.BufferedWriter; import java.io.IOException; import java.nio.charset.Charset; @@ -308,9 +310,9 @@ public void readImportManifestColumnListMustBeProvidedForGeneratedColumn() throw assertThat( e.getMessage(), equalTo( - "java.lang.RuntimeException: DB table table3 has one or more generated columns. An " - + "explict column list that excludes the generated columns must be provided in the " - + "manifest.")); + "java.lang.RuntimeException: DB table table3 has one or more generated columns. An" + + " explict column list that excludes the generated columns must be provided in" + + " the manifest.")); } } @@ -353,6 +355,76 @@ public void readImportManifestGeneratedColumnCannotBeImported() throws Exception } } + @Test + public void readPgImportManifestTypeMustMatch() throws Exception { + Path f11 = Files.createTempFile("table1-file", "1"); + Path manifestFile = Files.createTempFile("import-manifest", ".json"); + Charset charset = Charset.forName("UTF-8"); + try (BufferedWriter writer = Files.newBufferedWriter(manifestFile, charset)) { + String jsonString = + String.format( + "{\"tables\": [{\"table_name\": \"table1\",\"file_patterns\": [\"%s\"],\"columns\":" + + " [{\"column_name\": \"int_col\", \"type_name\": \"bigint\"}," + + " {\"column_name\":\"str_10_col\", \"type_name\": \"character varying(10)\"}," + + " {\"column_name\":\"float_col\", \"type_name\": \"double precision\"}," + + " {\"column_name\":\"bool_col\", \"type_name\": \"boolean\"}," + + " {\"column_name\": \"byte_col\", \"type_name\": \"bytea\"}," + + " {\"column_name\": \"timestamp_col\"," + + " \"type_name\":\"timestamp with time zone\"}," + + " {\"column_name\": \"numeric_col\", \"type_name\": \"numeric\"}," + + " {\"column_name\": \"date_col\", \"type_name\": \"date\"}]}]}", + f11.toString()); + writer.write(jsonString, 0, jsonString.length()); + } + + ValueProvider importManifest = + ValueProvider.StaticValueProvider.of(manifestFile.toString()); + PCollectionView ddlView = + pipeline.apply("ddl", Create.of(getPgTestDdl())).apply(View.asSingleton()); + + PCollection> tableAndFiles = + pipeline + .apply("Read manifest file", new ReadImportManifest(importManifest)) + .apply("Resolve data files", new ResolveDataFiles(importManifest, ddlView)); + pipeline.run(); + } + + @Test + public void readPgImportManifestTypeMismatch() throws Exception { + Path f11 = Files.createTempFile("table1-file", "1"); + Path manifestFile = Files.createTempFile("import-manifest", ".json"); + Charset charset = Charset.forName("UTF-8"); + try (BufferedWriter writer = Files.newBufferedWriter(manifestFile, charset)) { + String jsonString = + String.format( + "{\"tables\": [" + + "{\"table_name\": \"table1\"," + + "\"file_patterns\": [\"%s\"]," + + "\"columns\": [{\"column_name\": \"int_col\", \"type_name\": \"text\"}]}" + + "]}", + f11.toString()); + writer.write(jsonString, 0, jsonString.length()); + } + + ValueProvider importManifest = + ValueProvider.StaticValueProvider.of(manifestFile.toString()); + PCollectionView ddlView = + pipeline.apply("ddl", Create.of(getPgTestDdl())).apply(View.asSingleton()); + + PCollection> tableAndFiles = + pipeline + .apply("Read manifest file", new ReadImportManifest(importManifest)) + .apply("Resolve data files", new ResolveDataFiles(importManifest, ddlView)); + + PipelineExecutionException thrown = + assertThrows(PipelineExecutionException.class, () -> pipeline.run()); + assertThat( + thrown.getMessage(), + equalTo( + "java.lang.RuntimeException: Mismatching type: Table table1 Column int_col [PG_INT8" + + " from DB and text from manifest]")); + } + private static Ddl getTestDdl() { Ddl ddl = Ddl.builder() @@ -414,4 +486,42 @@ private static Ddl getTestDdl() { .build(); return ddl; } + + private static Ddl getPgTestDdl() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("table1") + .column("int_col") + .pgInt8() + .notNull() + .endColumn() + .column("str_10_col") + .pgVarchar() + .size(10) + .endColumn() + .column("float_col") + .pgFloat8() + .endColumn() + .column("bool_col") + .pgBool() + .endColumn() + .column("byte_col") + .pgBytea() + .endColumn() + .column("timestamp_col") + .pgTimestamptz() + .endColumn() + .column("numeric_col") + .pgNumeric() + .endColumn() + .column("date_col") + .pgDate() + .endColumn() + .primaryKey() + .asc("int_col") + .end() + .endTable() + .build(); + return ddl; + } } diff --git a/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java b/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java index 88630127dd..8f51002758 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java @@ -23,6 +23,7 @@ import com.google.cloud.teleport.spanner.TextImportProtos.ImportManifest.TableManifest; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -490,6 +491,107 @@ public void parseRowWithArrayColumn() throws Exception { pipeline.run(); } + @Test(expected = PipelineExecutionException.class) + public void parseRowWithPgArrayColumn() throws Exception { + PCollectionView ddlView = + pipeline.apply("ddl", Create.of(getTestPgDdlWithArray())).apply(View.asSingleton()); + PCollectionView>> tableColumnsMapView = + pipeline + .apply( + "tableColumnsMap", + Create.>>of(getEmptyTableColumnsMap()) + .withCoder( + MapCoder.of( + StringUtf8Coder.of(), + ListCoder.of(ProtoCoder.of(TableManifest.Column.class))))) + .apply("Map as view", View.asSingleton()); + + PCollection> input = + pipeline.apply("input", Create.of(KV.of(testTableName, "str, [a string in an array]"))); + PCollection mutations = + input.apply( + ParDo.of( + new TextRowToMutation( + ddlView, + tableColumnsMapView, + columnDelimiter, + StaticValueProvider.of('"'), + trailingDelimiter, + escape, + nullString, + dateFormat, + timestampFormat)) + .withSideInputs(ddlView, tableColumnsMapView)); + pipeline.run(); + } + + @Test + public void pgParseRowToMutation() { + PCollectionView ddlView = + pipeline.apply("ddl", Create.of(getPgTestDdl())).apply(View.asSingleton()); + PCollectionView>> tableColumnsMapView = + pipeline + .apply( + "tableColumnsMap", + Create.>>of(getEmptyTableColumnsMap()) + .withCoder( + MapCoder.of( + StringUtf8Coder.of(), + ListCoder.of(ProtoCoder.of(TableManifest.Column.class))))) + .apply("Map as view", View.asSingleton()); + + PCollection> input = + pipeline.apply( + "input", + Create.of( + KV.of( + testTableName, + "123,a string,'another" + + " string',1.23,True,2018-12-31T23:59:59Z,1567637083,aGk=," + + "-439.25335679, 1910-01-01"))); + PCollection mutations = + input.apply( + ParDo.of( + new TextRowToMutation( + ddlView, + tableColumnsMapView, + columnDelimiter, + StaticValueProvider.of('\''), + StaticValueProvider.of(true), + escape, + nullString, + dateFormat, + timestampFormat)) + .withSideInputs(ddlView, tableColumnsMapView)); + + PAssert.that(mutations) + .containsInAnyOrder( + Mutation.newInsertOrUpdateBuilder(testTableName) + .set("int_col") + .to(123) + .set("str_10_col") + .to("a string") + .set("str_max_col") + .to("another string") + .set("float_col") + .to(1.23) + .set("bool_col") + .to(true) + .set("timestamp_col") + .to(Value.timestamp(Timestamp.parseTimestamp("2018-12-31T23:59:59Z"))) + .set("timestamp_col_epoch") + .to(Value.timestamp(Timestamp.ofTimeMicroseconds(1567637083))) + .set("byte_col") + .to(Value.bytes(ByteArray.fromBase64("aGk="))) + .set("numeric_col") + .to("-439.25335679") + .set("date_col") + .to(Value.date(Date.parseDate("1910-01-01"))) + .build()); + + pipeline.run(); + } + private static Ddl getTestDdl() { Ddl ddl = Ddl.builder() @@ -538,6 +640,51 @@ private static Ddl getTestDdl() { return ddl; } + private static Ddl getPgTestDdl() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable(testTableName) + .column("int_col") + .pgInt8() + .notNull() + .endColumn() + .column("str_10_col") + .pgVarchar() + .size(10) + .endColumn() + .column("str_max_col") + .type(Type.pgVarchar()) + .max() + .endColumn() + .column("float_col") + .pgFloat8() + .endColumn() + .column("bool_col") + .pgBool() + .endColumn() + .column("timestamp_col") + .pgTimestamptz() + .endColumn() + .column("timestamp_col_epoch") + .pgTimestamptz() + .endColumn() + .column("byte_col") + .pgBytea() + .endColumn() + .column("numeric_col") + .pgNumeric() + .endColumn() + .column("date_col") + .pgDate() + .endColumn() + .primaryKey() + .asc("int_col") + .end() + .endTable() + .build(); + return ddl; + } + private static Ddl getTestDdlDateOnly() { Ddl ddl = Ddl.builder() @@ -589,6 +736,27 @@ private static Ddl getTestDdlWithArray() { return ddl; } + private static Ddl getTestPgDdlWithArray() { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable(testTableName) + .column("str_col") + .pgVarchar() + .max() + .notNull() + .endColumn() + .column("arr_str_col") + .type(Type.pgArray(Type.pgVarchar())) + .max() + .endColumn() + .primaryKey() + .asc("str_col") + .end() + .endTable() + .build(); + return ddl; + } + private static Map> getEmptyTableColumnsMap() { List columns = new ArrayList<>(); HashMap> tableColumnsMap = new HashMap<>(); diff --git a/src/test/java/com/google/cloud/teleport/templates/SpannerServerResource.java b/src/test/java/com/google/cloud/teleport/templates/SpannerServerResource.java index bd33a27549..1e99d6e139 100644 --- a/src/test/java/com/google/cloud/teleport/templates/SpannerServerResource.java +++ b/src/test/java/com/google/cloud/teleport/templates/SpannerServerResource.java @@ -19,6 +19,7 @@ import com.google.cloud.spanner.DatabaseAdminClient; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerException; import com.google.cloud.spanner.SpannerOptions; @@ -26,6 +27,7 @@ import com.google.cloud.spanner.TransactionRunner; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.RandomInsertMutationGenerator; +import java.util.Arrays; import java.util.Iterator; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.spanner.MutationGroup; @@ -56,7 +58,10 @@ protected void before() { SpannerOptions spannerOptions; if (EMULATOR_HOST == null) { spannerOptions = - SpannerOptions.newBuilder().setProjectId(this.projectId).setHost(host).build(); + SpannerOptions.newBuilder() + .setProjectId(this.projectId) + .setHost(host) + .build(); } else { spannerOptions = SpannerOptions.newBuilder() @@ -78,6 +83,20 @@ public void createDatabase(String dbName, Iterable ddlStatements) throws databaseAdminClient.createDatabase(this.instanceId, dbName, ddlStatements).get(); } + public void createPgDatabase(String dbName, Iterable ddlStatements) throws Exception { + databaseAdminClient + .createDatabase( + databaseAdminClient + .newDatabaseBuilder(DatabaseId.of(projectId, instanceId, dbName)) + .setDialect(Dialect.POSTGRESQL) + .build(), + Arrays.asList()) + .get(); + if (ddlStatements.iterator().hasNext()) { + databaseAdminClient.updateDatabaseDdl(instanceId, dbName, ddlStatements, null).get(); + } + } + public void updateDatabase(String dbName, Iterable ddlStatements) throws Exception { databaseAdminClient.updateDatabaseDdl(this.instanceId, dbName, ddlStatements, null).get(); } diff --git a/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java b/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java index 97bd419323..618ea4690e 100644 --- a/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java @@ -19,6 +19,7 @@ import com.google.cloud.teleport.spanner.IntegrationTest; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.templates.common.SpannerConverters; import com.google.cloud.teleport.templates.common.SpannerConverters.CreateTransactionFnWithTimestamp; import com.google.common.collect.Lists; @@ -48,6 +49,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptors; import org.junit.After; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,7 +66,7 @@ public final class SpannerToTextTest { private static final Logger LOG = LoggerFactory.getLogger(SpannerToTextTest.class); - static String tmpDir = Files.createTempDir().getAbsolutePath(); + private final String tmpDir = Files.createTempDir().getAbsolutePath(); private final Timestamp timestamp = new Timestamp(System.currentTimeMillis()); private final long numericTime = timestamp.getTime(); @@ -80,17 +82,29 @@ public final class SpannerToTextTest { @Rule public final SpannerServerResource spannerServer = new SpannerServerResource(); + @Before + public void setup() { + spannerServer.dropDatabase(sourceDb); + } + @After public void teardown() { spannerServer.dropDatabase(sourceDb); - spannerServer.dropDatabase(destDbPrefix + chkpt1); - spannerServer.dropDatabase(destDbPrefix + chkpt2); } /* Creates a database for a given Spanner database and populates it with * with random data */ private void createAndPopulate(String db, Ddl ddl, int numBatches) throws Exception { - spannerServer.createDatabase(db, ddl.statements()); + switch (ddl.dialect()) { + case GOOGLE_STANDARD_SQL: + spannerServer.createDatabase(db, ddl.statements()); + break; + case POSTGRESQL: + spannerServer.createPgDatabase(db, ddl.statements()); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + ddl.dialect()); + } spannerServer.populateRandomData(db, ddl, numBatches); } @@ -130,8 +144,7 @@ public void runExportWithTsTest() throws Exception { createAndPopulate(sourceDb, ddl, 100); // Export the database and note the timestamp ts1 - spannerServer.createDatabase(destDbPrefix + chkpt1, Collections.emptyList()); - exportDbAtTime(sourceDb, destDbPrefix + chkpt1, chkpt1, "", exportPipeline1); + exportDbAtTime(sourceDb, destDbPrefix + chkpt1, chkpt1, "", exportPipeline1, tmpDir); // Save the timestamp directly after the export String chkPt1Ts = getCurrentTimestamp(); @@ -141,10 +154,66 @@ public void runExportWithTsTest() throws Exception { // Add more records to the table, export the database and note the timestamp ts3 spannerServer.populateRandomData(sourceDb, ddl, 100); - spannerServer.createDatabase(destDbPrefix + chkpt2, Collections.emptyList()); // Export the table from the database using the saved timestamp - exportDbAtTime(sourceDb, destDbPrefix + chkpt2, chkpt2, chkPt1Ts, exportPipeline2); + exportDbAtTime(sourceDb, destDbPrefix + chkpt2, chkpt2, chkPt1Ts, exportPipeline2, tmpDir); + + File folder = new File(tmpDir + "/"); + + // Store the contents of directory containing the exported CSVs into a List + File[] files = folder.listFiles(); + + List oldData = readDbData(files, chkpt1); + List expectedOldData = readDbData(files, chkpt2); + + // Sort statements + Collections.sort(oldData); + Collections.sort(expectedOldData); + + assertEquals(oldData, expectedOldData); + } + + @Test + public void runPgExportWithTsTest() throws Exception { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable(tableName) + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("age") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first_name") + .desc("last_name") + .end() + .endTable() + .build(); + + /* Create initial table and populate + * numBatches = 100 + */ + createAndPopulate(sourceDb, ddl, 100); + + // Export the database and note the timestamp ts1 + exportDbAtTime(sourceDb, destDbPrefix + chkpt1, chkpt1, "", exportPipeline1, tmpDir); + + // Save the timestamp directly after the export + String chkPt1Ts = getCurrentTimestamp(); + + // Sleep for some time before adding more to the table + Thread.sleep(10000); + // Add more records to the table, export the database and note the timestamp ts3 + spannerServer.populateRandomData(sourceDb, ddl, 100); + + // Export the table from the database using the saved timestamp + exportDbAtTime(sourceDb, destDbPrefix + chkpt2, chkpt2, chkPt1Ts, exportPipeline2, tmpDir); File folder = new File(tmpDir + "/"); @@ -162,19 +231,25 @@ public void runExportWithTsTest() throws Exception { } private void exportDbAtTime( - String sourceDb, String destDb, String jobIdName, String ts, TestPipeline exportPipeline) { + String sourceDb, + String destDb, + String jobIdName, + String ts, + TestPipeline exportPipeline, + String outputDir) { ValueProvider.StaticValueProvider destination = - ValueProvider.StaticValueProvider.of(tmpDir + "/"); + ValueProvider.StaticValueProvider.of(outputDir + "/"); ValueProvider.StaticValueProvider jobId = ValueProvider.StaticValueProvider.of(jobIdName); ValueProvider.StaticValueProvider source = - ValueProvider.StaticValueProvider.of(tmpDir + "/" + jobIdName); + ValueProvider.StaticValueProvider.of(outputDir + "/" + jobIdName); ValueProvider.StaticValueProvider table = ValueProvider.StaticValueProvider.of(tableName); ValueProvider.StaticValueProvider timestamp = ValueProvider.StaticValueProvider.of(ts); ValueProvider.StaticValueProvider exportAsLogicalType = ValueProvider.StaticValueProvider.of(false); + SpannerConfig sourceConfig = spannerServer.getSpannerConfig(sourceDb); PCollectionView tx = diff --git a/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java b/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java index d366d68c86..31616b7392 100644 --- a/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java @@ -60,6 +60,8 @@ public class SpannerConverterTest implements Serializable { private static final String TABLE = "table"; private static final String COLUMN_NAME = "id"; + private static final String POSTGRESQL = "POSTGRESQL"; + private static final String GOOGLE_STANDARD_SQL = "GOOGLE_STANDARD_SQL"; @Rule public final transient TestPipeline pipeline = TestPipeline.create(); private SpannerConverters.StructCsvPrinter structCsvPrinter = @@ -91,9 +93,9 @@ public void testSchemaSave() throws IOException { when(databaseClient.readOnlyTransaction(tsbound)).thenReturn(readOnlyTransaction); when(readOnlyTransaction.executeQuery(any(Statement.class))).thenReturn(resultSet); - when(resultSet.next()).thenReturn(true).thenReturn(false); + when(resultSet.next()).thenReturn(true).thenReturn(true).thenReturn(false); when(resultSet.getCurrentRowAsStruct()).thenReturn(struct); - when(struct.getString(0)).thenReturn(COLUMN_NAME); + when(struct.getString(0)).thenReturn(GOOGLE_STANDARD_SQL).thenReturn(COLUMN_NAME); when(struct.getString(1)).thenReturn("INT64"); String schemaPath = "/tmp/" + UUID.randomUUID().toString(); @@ -118,6 +120,58 @@ public void testSchemaSave() throws IOException { assertEquals("{\"id\":\"INT64\"}", scanner.next()); } + @Test + @Category(NeedsRunner.class) + public void testPgSchemaSave() throws IOException { + + ValueProvider table = ValueProvider.StaticValueProvider.of(TABLE); + SpannerConfig spannerConfig = SpannerConfig.create().withDatabaseId("test-db"); + DatabaseClient databaseClient = mock(DatabaseClient.class, withSettings().serializable()); + ReadOnlyTransaction readOnlyTransaction = + mock(ReadOnlyTransaction.class, withSettings().serializable()); + ResultSet resultSet = mock(ResultSet.class, withSettings().serializable()); + Struct struct = mock(Struct.class, withSettings().serializable()); + + /* + * Get a second earlier than current time to avoid tests failing due to time mismatch across + * machines. A future timestamp is regarded as illegal when creating a timestamp bounded + * transaction. + */ + String instant = Instant.now().minus(1, ChronoUnit.SECONDS).toString(); + + ValueProvider.StaticValueProvider timestamp = + ValueProvider.StaticValueProvider.of(instant); + TimestampBound tsbound = getTimestampBound(instant); + + when(databaseClient.readOnlyTransaction(tsbound)).thenReturn(readOnlyTransaction); + when(readOnlyTransaction.executeQuery(any(Statement.class))).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true).thenReturn(true).thenReturn(false); + when(resultSet.getCurrentRowAsStruct()).thenReturn(struct); + when(struct.getString(0)).thenReturn(POSTGRESQL).thenReturn(COLUMN_NAME); + when(struct.getString(1)).thenReturn("bigint"); + + String schemaPath = "/tmp/" + UUID.randomUUID().toString(); + ValueProvider textWritePrefix = ValueProvider.StaticValueProvider.of(schemaPath); + SpannerConverters.ExportTransform exportTransform = + SpannerConverters.ExportTransformFactory.create( + table, spannerConfig, textWritePrefix, timestamp); + exportTransform.setDatabaseClient(databaseClient); + + PCollection results = pipeline.apply("Create", exportTransform); + ReadOperation readOperation = + ReadOperation.create() + .withQuery("SELECT \"id\" FROM \"table\";") + .withPartitionOptions(PartitionOptions.newBuilder().setMaxPartitions(1000).build()); + PAssert.that(results).containsInAnyOrder(readOperation); + pipeline.run(); + ReadableByteChannel channel = + FileSystems.open( + FileSystems.matchNewResource( + schemaPath + SpannerConverters.ExportTransform.ExportFn.SCHEMA_SUFFIX, false)); + java.util.Scanner scanner = new java.util.Scanner(channel).useDelimiter("\\A"); + assertEquals("{\"id\":\"bigint\"}", scanner.next()); + } + @Test public void testTwoFields() { assertEquals( From ac75e000f95425de7c797fc7ba0d18e1fc805a2d Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Wed, 11 May 2022 14:08:29 -0700 Subject: [PATCH 135/145] Import changes to support PG interface for Cloud Spanner PiperOrigin-RevId: 448080775 --- .../teleport/spanner/AvroRecordConverter.java | 5 ++- .../spanner/AvroSchemaToDdlConverter.java | 2 +- .../spanner/BuildReadFromTableOperations.java | 3 -- .../spanner/DdlToAvroSchemaConverter.java | 2 +- .../spanner/SpannerRecordConverter.java | 2 +- .../teleport/spanner/TextImportTransform.java | 4 +- .../teleport/spanner/TextRowToMutation.java | 4 +- .../cloud/teleport/spanner/common/Type.java | 4 +- .../teleport/spanner/ddl/CheckConstraint.java | 1 + .../cloud/teleport/spanner/ddl/Column.java | 1 + .../cloud/teleport/spanner/ddl/Ddl.java | 1 + .../spanner/ddl/DdlUtilityComponents.java | 1 + .../cloud/teleport/spanner/ddl/Dialect.java | 23 ----------- .../teleport/spanner/ddl/ForeignKey.java | 1 + .../cloud/teleport/spanner/ddl/Index.java | 1 + .../teleport/spanner/ddl/IndexColumn.java | 1 + .../spanner/ddl/InformationSchemaScanner.java | 1 + .../cloud/teleport/spanner/ddl/Table.java | 1 + .../cloud/teleport/spanner/ddl/View.java | 1 + .../templates/common/SpannerConverters.java | 39 ++++--------------- .../spanner/AvroSchemaToDdlConverterTest.java | 2 +- .../teleport/spanner/CopySchemaTest.java | 2 +- .../spanner/DdlToAvroSchemaConverterTest.java | 2 +- .../spanner/SpannerRecordConverterTest.java | 2 +- .../spanner/TextImportTransformTest.java | 2 +- .../spanner/TextRowToMutationTest.java | 10 +++-- .../cloud/teleport/spanner/ddl/DdlTest.java | 1 + .../ddl/InformationSchemaScannerTest.java | 25 ++++++++++++ .../spanner/ddl/RandomDdlGenerator.java | 1 + .../ddl/RandomInsertMutationGenerator.java | 1 + .../spanner/ddl/RandomValueGenerator.java | 10 ++++- .../teleport/templates/SpannerToTextTest.java | 39 +++++++++++++------ .../common/SpannerConverterTest.java | 35 +++++++++++++++-- 33 files changed, 137 insertions(+), 93 deletions(-) delete mode 100644 src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java b/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java index 167b04de06..6821026d8a 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroRecordConverter.java @@ -19,6 +19,7 @@ import com.google.cloud.Date; import com.google.cloud.Timestamp; import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Value; import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Table; @@ -113,7 +114,9 @@ public Mutation apply(GenericRecord record) { builder.set(column.name()).to(readNumeric(record, avroType, fieldName).orElse(null)); break; case PG_NUMERIC: - builder.set(column.name()).to(readPgNumeric(record, avroType, fieldName).orElse(null)); + builder + .set(column.name()) + .to(Value.pgNumeric(readPgNumeric(record, avroType, fieldName).orElse(null))); break; case ARRAY: case PG_ARRAY: diff --git a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java index cfd280c79c..aace53c20d 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java @@ -17,12 +17,12 @@ import static com.google.cloud.teleport.spanner.AvroUtil.unpackNullable; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.ddl.View; import com.google.common.base.Strings; diff --git a/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java b/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java index f20b823eab..523309959a 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java +++ b/src/main/java/com/google/cloud/teleport/spanner/BuildReadFromTableOperations.java @@ -137,9 +137,6 @@ private String createColumnExpression(Column col) { } return "t.`" + col.name() + "`"; case POSTGRESQL: - if (col.typeString().equals("numeric")) { - return "CAST(" + "t.\"" + col.name() + "\"" + " AS VARCHAR) AS " + col.name(); - } return "t.\"" + col.name() + "\""; default: throw new IllegalArgumentException( diff --git a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java index 50ce35ece0..474dada554 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java @@ -15,11 +15,11 @@ */ package com.google.cloud.teleport.spanner; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.IndexColumn; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.ddl.View; diff --git a/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java b/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java index 45f98b254f..8ed8b7873a 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java +++ b/src/main/java/com/google/cloud/teleport/spanner/SpannerRecordConverter.java @@ -16,9 +16,9 @@ package com.google.cloud.teleport.spanner; import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Struct; import com.google.cloud.teleport.spanner.common.NumericUtils; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.base.Strings; import java.nio.ByteBuffer; import java.util.List; diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java index 615907f036..84f1fe094d 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java @@ -15,13 +15,13 @@ */ package com.google.cloud.teleport.spanner; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Mutation; import com.google.cloud.teleport.spanner.TextImportProtos.ImportManifest; import com.google.cloud.teleport.spanner.TextImportProtos.ImportManifest.TableManifest; import com.google.cloud.teleport.spanner.common.Type.Code; import com.google.cloud.teleport.spanner.ddl.Column; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashMultimap; @@ -446,7 +446,7 @@ public static Code parseSpannerDataType(String columnType, Dialect dialect) { return Code.PG_TEXT; } else if (columnType.equalsIgnoreCase("date") && dialect == Dialect.POSTGRESQL) { return Code.PG_DATE; - }else { + } else { throw new IllegalArgumentException( "Unrecognized or unsupported column data type: " + columnType); } diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java b/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java index 8dcf9964cf..a0e4db173e 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextRowToMutation.java @@ -245,9 +245,11 @@ protected final Mutation parseRow( break; case NUMERIC: case JSON: - case PG_NUMERIC: columnValue = isNullValue ? Value.string(null) : Value.string(cellValue.trim()); break; + case PG_NUMERIC: + columnValue = isNullValue ? Value.pgNumeric(null) : Value.pgNumeric(cellValue.trim()); + break; case BYTES: case PG_BYTEA: columnValue = diff --git a/src/main/java/com/google/cloud/teleport/spanner/common/Type.java b/src/main/java/com/google/cloud/teleport/spanner/common/Type.java index 4da85725e4..58a3158a43 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/common/Type.java +++ b/src/main/java/com/google/cloud/teleport/spanner/common/Type.java @@ -15,7 +15,7 @@ */ package com.google.cloud.teleport.spanner.common; -import com.google.cloud.teleport.spanner.ddl.Dialect; +import com.google.cloud.spanner.Dialect; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -223,7 +223,7 @@ public static Type pgArray(Type elementType) { case PG_DATE: return TYPE_PG_ARRAY_DATE; default: - return new Type(Code.PG_ARRAY, elementType, null); + throw new IllegalArgumentException("Unknown Array type: Array of " + elementType); } } diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java index 2a9570dd1e..6b5ba4ba11 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/CheckConstraint.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import java.io.IOException; import java.io.Serializable; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java index 4658240274..27fa75b9a8 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Column.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.common.Type; import com.google.common.collect.ImmutableList; import java.io.IOException; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java index 600d2d4e31..8ede2e71cc 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner.ddl; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.ExportProtos.Export; import com.google.common.base.Function; import com.google.common.collect.Collections2; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java index 8827e28dec..d47d6bec30 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/DdlUtilityComponents.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner.ddl; +import com.google.cloud.spanner.Dialect; import com.google.common.escape.Escaper; import com.google.common.escape.Escapers; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java deleted file mode 100644 index 8a2ed8f3ff..0000000000 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Dialect.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Copyright 2021 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.ddl; - -/** Describe the dialect of the Cloud Spanner Database. */ -public enum Dialect { - GOOGLE_STANDARD_SQL, - POSTGRESQL; -} diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java index dd727a7c17..8bb4c172f4 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/ForeignKey.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.Serializable; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java index 7c8ea8729f..42d3c635a6 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Index.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.Serializable; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java index bc263046a6..ce4e150a1c 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/IndexColumn.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.Serializable; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java index c2d5b13075..c435c458a6 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner.ddl; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.ReadContext; import com.google.cloud.spanner.ResultSet; import com.google.cloud.spanner.Statement; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java index d8208867bd..bf713a890f 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/Table.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import java.io.IOException; diff --git a/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java b/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java index d724ee2778..b03d6287dd 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ddl/View.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import java.io.IOException; import java.io.Serializable; import javax.annotation.Nullable; diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java index 636f1aab64..63605eb433 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java @@ -20,6 +20,7 @@ import com.google.cloud.Timestamp; import com.google.cloud.spanner.BatchReadOnlyTransaction; import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.PartitionOptions; import com.google.cloud.spanner.ReadContext; import com.google.cloud.spanner.ReadOnlyTransaction; @@ -30,7 +31,6 @@ import com.google.cloud.spanner.Type; import com.google.cloud.spanner.Type.Code; import com.google.cloud.spanner.Type.StructField; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -229,10 +229,10 @@ public void processElement(ProcessContext processContext) { DatabaseClient databaseClient = getDatabaseClient(spannerConfig()); String timestampString = this.timestamp.get(); TimestampBound tsbound = getTimestampBound(timestampString); + LOG.info("Reading dialect information"); + dialect = databaseClient.getDialect(); try (ReadOnlyTransaction context = databaseClient.readOnlyTransaction(tsbound)) { - LOG.info("Reading dialect information"); - dialect = getDialect(context); LOG.info("Reading schema information"); columns = getAllColumns(context, table().get(), dialect); String columnJson = SpannerConverters.GSON.toJson(columns); @@ -274,9 +274,6 @@ public void processElement(ProcessContext processContext) { private String createColumnExpression(String columnName, String columnType, Dialect dialect) { if (dialect == Dialect.POSTGRESQL) { - if (columnType.equals("numeric")) { - return "CAST(\"" + columnName + "\" AS VARCHAR) AS " + columnName; - } return "\"" + columnName + "\""; } if (columnType.equals("NUMERIC")) { @@ -335,17 +332,12 @@ private LinkedHashMap getAllColumns( break; case POSTGRESQL: statement = - "SELECT COLUMN_NAME, SPANNER_TYPE FROM INFORMATION_SCHEMA.COLUMNS WHERE" - + " TABLE_NAME=$1 AND TABLE_CATALOG=$2 AND TABLE_SCHEMA='public'" + "SELECT COLUMN_NAME, SPANNER_TYPE FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME=$1" + + " AND TABLE_SCHEMA NOT IN ('information_schema', 'spanner_sys', 'pg_catalog')" + " AND IS_GENERATED = 'NEVER' ORDER BY ORDINAL_POSITION;"; resultSet = context.executeQuery( - Statement.newBuilder(statement) - .bind("p1") - .to(tableName) - .bind("p2") - .to(spannerConfig().getDatabaseId().get()) - .build()); + Statement.newBuilder(statement).bind("p1").to(tableName).build()); break; default: throw new IllegalArgumentException(String.format("Unrecognized dialect: %s", dialect)); @@ -357,23 +349,6 @@ private LinkedHashMap getAllColumns( } return columns; } - - private Dialect getDialect(ReadContext context) { - String dialect = ""; - Statement statement = - Statement.of( - "SELECT CASE schema_name WHEN 'information_schema' THEN 'POSTGRESQL' WHEN" - + " 'INFORMATION_SCHEMA' THEN 'GOOGLE_STANDARD_SQL' END dialect FROM" - + " information_schema.schemata WHERE schema_name IN('information_schema'," - + " 'INFORMATION_SCHEMA');"); - ResultSet resultSet = context.executeQuery(statement); - LOG.info("Got dialect information."); - if (resultSet.next()) { - Struct currentRow = resultSet.getCurrentRowAsStruct(); - dialect = currentRow.getString(0); - } - return Dialect.valueOf(dialect); - } } /** @@ -463,6 +438,7 @@ private static BiFunction getColumnParser(Type.Code colu return nullSafeColumnParser( ((currentRow, columnName) -> Double.toString(currentRow.getDouble(columnName)))); case STRING: + case PG_NUMERIC: return nullSafeColumnParser(Struct::getString); case BYTES: return nullSafeColumnParser( @@ -496,6 +472,7 @@ private static String parseArrayValue(Struct currentRow, String columnName) { case FLOAT64: return GSON.toJson(currentRow.getDoubleArray(columnName)); case STRING: + case PG_NUMERIC: return GSON.toJson(currentRow.getStringList(columnName)); case BYTES: return GSON.toJson( diff --git a/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java index 5ba2eef970..2da05cb682 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java @@ -21,8 +21,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; diff --git a/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java b/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java index 306c916c30..96f531aed3 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/CopySchemaTest.java @@ -18,8 +18,8 @@ import static org.hamcrest.text.IsEqualCompressingWhiteSpace.equalToCompressingWhiteSpace; import static org.junit.Assert.assertThat; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.RandomDdlGenerator; import java.io.IOException; import java.util.Collection; diff --git a/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java index 0c1b615c6d..07ea2158e2 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java @@ -22,10 +22,10 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.spanner.ddl.View; import com.google.common.collect.ImmutableList; import java.util.Collection; diff --git a/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java b/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java index fb813826dc..072ca2d348 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/SpannerRecordConverterTest.java @@ -22,11 +22,11 @@ import com.google.cloud.ByteArray; import com.google.cloud.Date; import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Struct; import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.common.collect.Lists; import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java b/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java index 7961888e3a..78727cef60 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/TextImportTransformTest.java @@ -19,10 +19,10 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThrows; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.TextImportTransform.ReadImportManifest; import com.google.cloud.teleport.spanner.TextImportTransform.ResolveDataFiles; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import java.io.BufferedWriter; import java.io.IOException; import java.nio.charset.Charset; diff --git a/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java b/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java index 8f51002758..d1be15fd4f 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/TextRowToMutationTest.java @@ -15,15 +15,17 @@ */ package com.google.cloud.teleport.spanner; +import static org.junit.Assert.assertThrows; + import com.google.cloud.ByteArray; import com.google.cloud.Date; import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Value; import com.google.cloud.teleport.spanner.TextImportProtos.ImportManifest.TableManifest; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -491,7 +493,7 @@ public void parseRowWithArrayColumn() throws Exception { pipeline.run(); } - @Test(expected = PipelineExecutionException.class) + @Test public void parseRowWithPgArrayColumn() throws Exception { PCollectionView ddlView = pipeline.apply("ddl", Create.of(getTestPgDdlWithArray())).apply(View.asSingleton()); @@ -522,7 +524,7 @@ public void parseRowWithPgArrayColumn() throws Exception { dateFormat, timestampFormat)) .withSideInputs(ddlView, tableColumnsMapView)); - pipeline.run(); + assertThrows(PipelineExecutionException.class, () -> pipeline.run()); } @Test @@ -584,7 +586,7 @@ public void pgParseRowToMutation() { .set("byte_col") .to(Value.bytes(ByteArray.fromBase64("aGk="))) .set("numeric_col") - .to("-439.25335679") + .to(Value.pgNumeric("-439.25335679")) .set("date_col") .to(Value.date(Date.parseDate("1910-01-01"))) .build()); diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index b0602531a5..49a04428ed 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.ExportProtos.Export; import com.google.cloud.teleport.spanner.common.Type; import com.google.common.collect.ImmutableList; diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java index e6fb22ea9c..aa053f14b3 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerTest.java @@ -26,6 +26,7 @@ import com.google.cloud.spanner.BatchClient; import com.google.cloud.spanner.BatchReadOnlyTransaction; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.TimestampBound; import com.google.cloud.teleport.spanner.IntegrationTest; import com.google.cloud.teleport.spanner.SpannerServerResource; @@ -274,6 +275,30 @@ public void simpleView() throws Exception { assertThat(view.query(), equalTo("SELECT u.name FROM Users u")); } + @Test + public void pgSimpleView() throws Exception { + String tableDef = + "CREATE TABLE \"Users\" (" + + " id bigint NOT NULL," + + " name character varying," + + " PRIMARY KEY (id)) "; + String viewDef = "CREATE VIEW \"Names\" SQL SECURITY INVOKER AS SELECT u.name FROM \"Users\" u"; + + spannerServer.createPgDatabase(dbId, Arrays.asList(tableDef, viewDef)); + Ddl ddl = getPgDatabaseDdl(); + + assertThat(ddl.allTables(), hasSize(1)); + assertThat(ddl.table("Users"), notNullValue()); + assertThat(ddl.table("uSers"), notNullValue()); + + assertThat(ddl.views(), hasSize(1)); + View view = ddl.view("Names"); + assertThat(view, notNullValue()); + assertThat(ddl.view("nAmes"), sameInstance(view)); + + assertThat(view.query(), equalTo("SELECT u.name FROM \"Users\" u")); + } + @Test public void interleavedIn() throws Exception { List statements = diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java index 2a5d5b7a43..34c8a5cd97 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomDdlGenerator.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.spanner.ddl; import com.google.auto.value.AutoValue; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.common.Type; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java index 4e351dd899..b1a1c88aa2 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomInsertMutationGenerator.java @@ -20,6 +20,7 @@ import com.google.cloud.ByteArray; import com.google.cloud.Date; import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Value; import java.math.BigDecimal; diff --git a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java index e294713747..04055d13ad 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ddl/RandomValueGenerator.java @@ -52,13 +52,13 @@ public Stream valueStream(Column column, boolean notNull) { threshold = -1; } if (random.nextInt(100) < threshold) { - return generateNullValue(column.type()); + return generateNullOrNaNValue(column.type()); } return generate(column); }); } - private Value generateNullValue(Type type) { + private Value generateNullOrNaNValue(Type type) { switch (type.getCode()) { case BOOL: case PG_BOOL: @@ -68,6 +68,9 @@ private Value generateNullValue(Type type) { return Value.int64(null); case FLOAT64: case PG_FLOAT8: + if (random.nextBoolean()) { + return Value.float64(Double.NaN); + } return Value.float64(null); case BYTES: case PG_BYTEA: @@ -85,6 +88,9 @@ private Value generateNullValue(Type type) { case NUMERIC: return Value.numeric(null); case PG_NUMERIC: + if (random.nextBoolean()) { + return Value.pgNumeric("NaN"); + } return Value.pgNumeric(null); case ARRAY: case PG_ARRAY: diff --git a/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java b/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java index 618ea4690e..8d6be81ec5 100644 --- a/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/SpannerToTextTest.java @@ -17,9 +17,11 @@ import static org.junit.Assert.assertEquals; +import com.google.cloud.spanner.Dialect; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.Statement; import com.google.cloud.teleport.spanner.IntegrationTest; import com.google.cloud.teleport.spanner.ddl.Ddl; -import com.google.cloud.teleport.spanner.ddl.Dialect; import com.google.cloud.teleport.templates.common.SpannerConverters; import com.google.cloud.teleport.templates.common.SpannerConverters.CreateTransactionFnWithTimestamp; import com.google.common.collect.Lists; @@ -29,7 +31,6 @@ import java.io.FileReader; import java.io.IOException; import java.sql.Timestamp; -import java.time.Instant; import java.util.Collections; import java.util.List; import org.apache.beam.sdk.io.TextIO; @@ -108,9 +109,28 @@ private void createAndPopulate(String db, Ddl ddl, int numBatches) throws Except spannerServer.populateRandomData(db, ddl, numBatches); } - String getCurrentTimestamp() { - Instant instant = Instant.now(); - return instant.toString(); + String getCurrentTimestamp(Dialect dialect) { + String sql; + switch (dialect) { + case GOOGLE_STANDARD_SQL: + sql = "SELECT CURRENT_TIMESTAMP();"; + break; + case POSTGRESQL: + sql = "SELECT now();"; + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + dialect); + } + String timestamp; + try (ResultSet resultSet = + spannerServer + .getDbClient(sourceDb) + .singleUseReadOnlyTransaction() + .executeQuery(Statement.of(sql))) { + resultSet.next(); + timestamp = resultSet.getTimestamp(0).toString(); + } + return timestamp; } /* Validates behavior of database export without specifying timestamp @@ -147,10 +167,7 @@ public void runExportWithTsTest() throws Exception { exportDbAtTime(sourceDb, destDbPrefix + chkpt1, chkpt1, "", exportPipeline1, tmpDir); // Save the timestamp directly after the export - String chkPt1Ts = getCurrentTimestamp(); - - // Sleep for some time before adding more to the table - Thread.sleep(10000); + String chkPt1Ts = getCurrentTimestamp(Dialect.GOOGLE_STANDARD_SQL); // Add more records to the table, export the database and note the timestamp ts3 spannerServer.populateRandomData(sourceDb, ddl, 100); @@ -205,10 +222,8 @@ public void runPgExportWithTsTest() throws Exception { exportDbAtTime(sourceDb, destDbPrefix + chkpt1, chkpt1, "", exportPipeline1, tmpDir); // Save the timestamp directly after the export - String chkPt1Ts = getCurrentTimestamp(); + String chkPt1Ts = getCurrentTimestamp(Dialect.POSTGRESQL); - // Sleep for some time before adding more to the table - Thread.sleep(10000); // Add more records to the table, export the database and note the timestamp ts3 spannerServer.populateRandomData(sourceDb, ddl, 100); diff --git a/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java b/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java index 31616b7392..7999d7bbc8 100644 --- a/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java +++ b/src/test/java/com/google/cloud/teleport/templates/common/SpannerConverterTest.java @@ -26,6 +26,7 @@ import com.google.cloud.Date; import com.google.cloud.Timestamp; import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.PartitionOptions; import com.google.cloud.spanner.ReadOnlyTransaction; import com.google.cloud.spanner.ResultSet; @@ -60,8 +61,8 @@ public class SpannerConverterTest implements Serializable { private static final String TABLE = "table"; private static final String COLUMN_NAME = "id"; - private static final String POSTGRESQL = "POSTGRESQL"; - private static final String GOOGLE_STANDARD_SQL = "GOOGLE_STANDARD_SQL"; + private static final Dialect POSTGRESQL = Dialect.POSTGRESQL; + private static final Dialect GOOGLE_STANDARD_SQL = Dialect.GOOGLE_STANDARD_SQL; @Rule public final transient TestPipeline pipeline = TestPipeline.create(); private SpannerConverters.StructCsvPrinter structCsvPrinter = @@ -95,8 +96,9 @@ public void testSchemaSave() throws IOException { when(readOnlyTransaction.executeQuery(any(Statement.class))).thenReturn(resultSet); when(resultSet.next()).thenReturn(true).thenReturn(true).thenReturn(false); when(resultSet.getCurrentRowAsStruct()).thenReturn(struct); - when(struct.getString(0)).thenReturn(GOOGLE_STANDARD_SQL).thenReturn(COLUMN_NAME); + when(struct.getString(0)).thenReturn(COLUMN_NAME); when(struct.getString(1)).thenReturn("INT64"); + when(databaseClient.getDialect()).thenReturn(GOOGLE_STANDARD_SQL); String schemaPath = "/tmp/" + UUID.randomUUID().toString(); ValueProvider textWritePrefix = ValueProvider.StaticValueProvider.of(schemaPath); @@ -147,8 +149,9 @@ public void testPgSchemaSave() throws IOException { when(readOnlyTransaction.executeQuery(any(Statement.class))).thenReturn(resultSet); when(resultSet.next()).thenReturn(true).thenReturn(true).thenReturn(false); when(resultSet.getCurrentRowAsStruct()).thenReturn(struct); - when(struct.getString(0)).thenReturn(POSTGRESQL).thenReturn(COLUMN_NAME); + when(struct.getString(0)).thenReturn(COLUMN_NAME); when(struct.getString(1)).thenReturn("bigint"); + when(databaseClient.getDialect()).thenReturn(POSTGRESQL); String schemaPath = "/tmp/" + UUID.randomUUID().toString(); ValueProvider textWritePrefix = ValueProvider.StaticValueProvider.of(schemaPath); @@ -255,6 +258,17 @@ public void testTimestamp() { .build())); } + @Test + public void testPgNumeric() { + assertEquals( + "\"-25398514232141142.0014578090\"", + structCsvPrinter.print( + Struct.newBuilder() + .set("col") + .to(Value.pgNumeric("-25398514232141142.0014578090")) + .build())); + } + @Test public void testBooleanArray() { assertEquals( @@ -353,4 +367,17 @@ public void testBytesArray() { .to(Value.bytesArray(Collections.singletonList(ByteArray.copyFrom("test")))) .build())); } + + @Test + public void testPgNumericArray() { + assertEquals( + "\"[\"\"-25398514232141142.0014578090\"\"]\"", + structCsvPrinter.print( + Struct.newBuilder() + .set("col") + .to( + Value.pgNumericArray( + Collections.singletonList("-25398514232141142.0014578090"))) + .build())); + } } From 272f641e738f96fce1410cfca79fff3f08f7c83d Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Wed, 11 May 2022 16:33:57 -0700 Subject: [PATCH 136/145] Support for Automatic Dialect Detection and Integration tests PiperOrigin-RevId: 448112311 --- .../teleport/spanner/ExportTransform.java | 33 +- .../teleport/spanner/ImportTransform.java | 72 ++- .../cloud/teleport/spanner/ReadDialect.java | 69 +++ .../spanner/ReadInformationSchema.java | 21 +- .../teleport/spanner/TextImportTransform.java | 74 ++- .../templates/common/SpannerConverters.java | 1 + src/main/proto/export.proto | 7 + src/main/proto/text_import.proto | 5 + .../teleport/spanner/CompareDatabases.java | 13 +- .../cloud/teleport/spanner/CopyDbTest.java | 476 ++++++++++++++- .../teleport/spanner/ExportTimestampTest.java | 179 +++++- .../teleport/spanner/ExportTransformTest.java | 10 +- .../teleport/spanner/ImportFromAvroTest.java | 549 +++++++++++++++++- .../spanner/TextImportTransformTest.java | 2 +- .../ddl/InformationSchemaScannerTest.java | 20 +- .../spanner/ddl/RandomDdlGenerator.java | 7 + .../ddl/RandomInsertMutationGenerator.java | 4 + 17 files changed, 1466 insertions(+), 76 deletions(-) create mode 100644 src/main/java/com/google/cloud/teleport/spanner/ReadDialect.java diff --git a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java index be55b9e35c..d4d04724a0 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java @@ -18,9 +18,11 @@ import static com.google.cloud.teleport.spanner.SpannerTableFilter.getFilteredTables; import static com.google.cloud.teleport.util.ValueProviderUtils.eitherOrValueProvider; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.TimestampBound; import com.google.cloud.teleport.spanner.ExportProtos.Export; +import com.google.cloud.teleport.spanner.ExportProtos.ProtoDialect; import com.google.cloud.teleport.spanner.ExportProtos.TableManifest; import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Ddl; @@ -187,10 +189,15 @@ public WriteFilesResult expand(PBegin begin) { .apply( "Create transaction", ParDo.of(new CreateTransactionFnWithTimestamp(spannerConfig, snapshotTime))) - .apply("As PCollectionView", View.asSingleton()); + .apply("Tx As PCollectionView", View.asSingleton()); + + PCollectionView dialectView = + p.apply("Read Dialect", new ReadDialect(spannerConfig)) + .apply("Dialect As PCollectionView", View.asSingleton()); PCollection ddl = - p.apply("Read Information Schema", new ReadInformationSchema(spannerConfig, tx)); + p.apply( + "Read Information Schema", new ReadInformationSchema(spannerConfig, tx, dialectView)); PCollection exportState = ddl.apply( @@ -377,7 +384,9 @@ public void processElement(ProcessContext c) { rows.apply( "Store Avro files", AvroIO.writeCustomTypeToGenericRecords() - .to(new SchemaBasedDynamicDestinations(avroSchemas, outputDirectoryName, resource)) + .to( + new SchemaBasedDynamicDestinations( + avroSchemas, outputDirectoryName, dialectView, resource)) .withTempDirectory(tempResource)); // Generate the manifest file. @@ -558,7 +567,8 @@ private OutputStream createOutputStream(Path outputPath, ProcessContext c) PCollection metadataContent = metadataTables.apply( "Create database manifest", - ParDo.of(new CreateDatabaseManifest(ddlView)).withSideInputs(ddlView)); + ParDo.of(new CreateDatabaseManifest(ddlView, dialectView)) + .withSideInputs(ddlView, dialectView)); Contextful.Fn manifestNaming = (element, c) -> @@ -584,14 +594,17 @@ private static class SchemaBasedDynamicDestinations private final PCollectionView> avroSchemas; private final PCollectionView uniqueIdView; + private final PCollectionView dialectView; private final ValueProvider baseDir; private SchemaBasedDynamicDestinations( PCollectionView> avroSchemas, PCollectionView uniqueIdView, + PCollectionView dialectView, ValueProvider baseDir) { this.avroSchemas = avroSchemas; this.uniqueIdView = uniqueIdView; + this.dialectView = dialectView; this.baseDir = baseDir; } @@ -638,14 +651,15 @@ public FileBasedSink.FilenamePolicy getFilenamePolicy(final String destination) @Override public List> getSideInputs() { - return Arrays.asList(avroSchemas, uniqueIdView); + return Arrays.asList(avroSchemas, uniqueIdView, dialectView); } @Override public GenericRecord formatRecord(Struct record) { String table = record.getString(0); Schema schema = sideInput(avroSchemas).get(table).get(); - return new SpannerRecordConverter(schema).convert(record); + Dialect dialect = sideInput(dialectView); + return new SpannerRecordConverter(schema, dialect).convert(record); } } @@ -740,15 +754,19 @@ public List extractOutput(List accumulator) { static class CreateDatabaseManifest extends DoFn, String> { private final PCollectionView ddlView; + private final PCollectionView dialectView; - public CreateDatabaseManifest(PCollectionView ddlView) { + public CreateDatabaseManifest( + PCollectionView ddlView, PCollectionView dialectView) { this.ddlView = ddlView; + this.dialectView = dialectView; } @ProcessElement public void processElement( @Element List exportMetadata, OutputReceiver out, ProcessContext c) { Ddl ddl = c.sideInput(ddlView); + Dialect dialect = c.sideInput(dialectView); ExportProtos.Export.Builder exportManifest = ExportProtos.Export.newBuilder(); for (Export.Table obj : exportMetadata) { if (ddl.changeStream(obj.getName()) != null) { @@ -758,6 +776,7 @@ public void processElement( } } exportManifest.addAllDatabaseOptions(ddl.databaseOptions()); + exportManifest.setDialect(ProtoDialect.valueOf(dialect.name())); try { out.output(JsonFormat.printer().print(exportManifest.build())); } catch (InvalidProtocolBufferException e) { diff --git a/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java index d532efd993..dbc4df4d59 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java @@ -17,8 +17,10 @@ import com.google.api.gax.longrunning.OperationFuture; import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Mutation; import com.google.cloud.teleport.spanner.ExportProtos.Export; +import com.google.cloud.teleport.spanner.ExportProtos.ProtoDialect; import com.google.cloud.teleport.spanner.ExportProtos.TableManifest; import com.google.cloud.teleport.spanner.ddl.ChangeStream; import com.google.cloud.teleport.spanner.ddl.Ddl; @@ -124,8 +126,14 @@ public ImportTransform( @Override public PDone expand(PBegin begin) { + PCollectionView dialectView = + begin + .apply("Read Dialect", new ReadDialect(spannerConfig)) + .apply("Dialect As PCollectionView", View.asSingleton()); + PCollection manifest = - begin.apply("Read manifest", new ReadExportManifestFile(importDirectory)); + begin.apply("Read manifest", new ReadExportManifestFile(importDirectory, dialectView)); + PCollectionView manifestView = manifest.apply("Manifest as view", View.asSingleton()); PCollection> allFiles = @@ -157,7 +165,8 @@ public void processElement(ProcessContext c) { begin.apply(SpannerIO.createTransaction().withSpannerConfig(spannerConfig)); PCollection informationSchemaDdl = - begin.apply("Read Information Schema", new ReadInformationSchema(spannerConfig, tx)); + begin.apply( + "Read Information Schema", new ReadInformationSchema(spannerConfig, tx, dialectView)); final PCollectionView>> avroDdlView = avroSchemas.apply("Avro ddl view", View.asSingleton()); @@ -259,7 +268,8 @@ public void processElement(ProcessContext c) { .withCommitDeadline(Duration.standardMinutes(1)) .withMaxCumulativeBackoff(Duration.standardHours(2)) .withMaxNumMutations(10000) - .withGroupingFactor(100)); + .withGroupingFactor(100) + .withDialectView(dialectView)); previousComputation = result.getOutput(); } ddl.apply(Wait.on(previousComputation)) @@ -278,25 +288,51 @@ public void processElement(ProcessContext c) { private static class ReadExportManifestFile extends PTransform> { private final ValueProvider importDirectory; + private final PCollectionView dialectView; - private ReadExportManifestFile(ValueProvider importDirectory) { + private ReadExportManifestFile( + ValueProvider importDirectory, PCollectionView dialectView) { this.importDirectory = importDirectory; + this.dialectView = dialectView; } @Override public PCollection expand(PBegin input) { NestedValueProvider manifestFile = NestedValueProvider.of(importDirectory, s -> GcsUtil.joinPath(s, "spanner-export.json")); - return input - .apply("Read manifest", FileIO.match().filepattern(manifestFile)) - .apply( - "Resource id", - MapElements.into(TypeDescriptor.of(ResourceId.class)) - .via((MatchResult.Metadata::resourceId))) - .apply( - "Read manifest json", - MapElements.into(TypeDescriptor.of(Export.class)) - .via(ReadExportManifestFile::readManifest)); + PCollection manifest = + input + .apply("Read manifest", FileIO.match().filepattern(manifestFile)) + .apply( + "Resource id", + MapElements.into(TypeDescriptor.of(ResourceId.class)) + .via((MatchResult.Metadata::resourceId))) + .apply( + "Read manifest json", + MapElements.into(TypeDescriptor.of(Export.class)) + .via(ReadExportManifestFile::readManifest)); + manifest.apply( + "Check dialect", + ParDo.of( + new DoFn() { + + @ProcessElement + public void processElement(ProcessContext c) { + Export proto = c.element(); + Dialect dialect = c.sideInput(dialectView); + ProtoDialect protoDialect = proto.getDialect(); + if (!protoDialect.name().equals(dialect.name())) { + throw new RuntimeException( + String.format( + "Dialect mismatches: Dialect of the database (%s) is different from" + + " the one in exported manifest (%s).", + dialect, protoDialect)); + } + c.output(dialect); + } + }) + .withSideInputs(dialectView)); + return manifest; } private static Export readManifest(ResourceId fileResource) { @@ -426,6 +462,7 @@ public void teardown() { public void processElement(ProcessContext c) { List> avroSchemas = c.sideInput(avroSchemasView); Ddl informationSchemaDdl = c.sideInput(informationSchemaView); + Dialect dialect = informationSchemaDdl.dialect(); Export manifest = c.sideInput(manifestView); if (LOG.isDebugEnabled()) { @@ -449,7 +486,8 @@ public void processElement(ProcessContext c) { } } } - AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(); + AvroSchemaToDdlConverter converter = + new AvroSchemaToDdlConverter(dialect); List createIndexStatements = new ArrayList<>(); List createForeignKeyStatements = new ArrayList<>(); List createChangeStreamStatements = new ArrayList<>(); @@ -458,7 +496,7 @@ public void processElement(ProcessContext c) { List ddlStatements = new ArrayList<>(); if (!manifest.getDatabaseOptionsList().isEmpty()) { - Ddl.Builder builder = Ddl.builder(); + Ddl.Builder builder = Ddl.builder(dialect); builder.mergeDatabaseOptions(manifest.getDatabaseOptionsList()); mergedDdl.mergeDatabaseOptions(manifest.getDatabaseOptionsList()); Ddl newDdl = builder.build(); @@ -467,7 +505,7 @@ public void processElement(ProcessContext c) { } if (!missingTables.isEmpty() || !missingViews.isEmpty()) { - Ddl.Builder builder = Ddl.builder(); + Ddl.Builder builder = Ddl.builder(dialect); for (KV kv : missingViews) { com.google.cloud.teleport.spanner.ddl.View view = converter.toView(kv.getKey(), kv.getValue()); diff --git a/src/main/java/com/google/cloud/teleport/spanner/ReadDialect.java b/src/main/java/com/google/cloud/teleport/spanner/ReadDialect.java new file mode 100644 index 0000000000..7050b7ef46 --- /dev/null +++ b/src/main/java/com/google/cloud/teleport/spanner/ReadDialect.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2022 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.DatabaseClient; +import com.google.cloud.spanner.Dialect; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; + +/** Retrieve {@link Dialect} from Spanner database for further usage. */ +public class ReadDialect extends PTransform> { + + private final SpannerConfig spannerConfig; + + public ReadDialect(SpannerConfig spannerConfig) { + this.spannerConfig = spannerConfig; + } + + @Override + public PCollection expand(PBegin p) { + return p.apply("Create empty", Create.of((Void) null)) + .apply("Read dialect from Spanner", ParDo.of(new ReadDialectFn(spannerConfig))); + } + + private static class ReadDialectFn extends DoFn { + private final SpannerConfig spannerConfig; + private transient SpannerAccessor spannerAccessor; + + public ReadDialectFn(SpannerConfig spannerConfig) { + this.spannerConfig = spannerConfig; + } + + @Setup + public void setup() throws Exception { + spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); + } + + @Teardown + public void teardown() throws Exception { + spannerAccessor.close(); + } + + @ProcessElement + public void processElement(ProcessContext c) { + DatabaseClient databaseClient = spannerAccessor.getDatabaseClient(); + Dialect dialect = databaseClient.getDialect(); + c.output(dialect); + } + } +} diff --git a/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java b/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java index 4360d1c733..2b3bc5fe19 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java +++ b/src/main/java/com/google/cloud/teleport/spanner/ReadInformationSchema.java @@ -18,6 +18,7 @@ import com.google.cloud.spanner.BatchClient; import com.google.cloud.spanner.BatchReadOnlyTransaction; import com.google.cloud.spanner.BatchTransactionId; +import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.InformationSchemaScanner; import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; @@ -36,10 +37,15 @@ public class ReadInformationSchema extends PTransform> private final SpannerConfig spannerConfig; private final PCollectionView tx; + private final PCollectionView dialectView; - public ReadInformationSchema(SpannerConfig spannerConfig, PCollectionView tx) { + public ReadInformationSchema( + SpannerConfig spannerConfig, + PCollectionView tx, + PCollectionView dialectView) { this.spannerConfig = spannerConfig; this.tx = tx; + this.dialectView = dialectView; } @Override @@ -47,17 +53,23 @@ public PCollection expand(PBegin p) { return p.apply("Create empty", Create.of((Void) null)) .apply( "Read Information Schema", - ParDo.of(new ReadInformationSchemaFn(spannerConfig, tx)).withSideInputs(tx)); + ParDo.of(new ReadInformationSchemaFn(spannerConfig, tx, dialectView)) + .withSideInputs(tx, dialectView)); } private static class ReadInformationSchemaFn extends DoFn { private final SpannerConfig spannerConfig; private transient SpannerAccessor spannerAccessor; private final PCollectionView tx; + private final PCollectionView dialectView; - public ReadInformationSchemaFn(SpannerConfig spannerConfig, PCollectionView tx) { + public ReadInformationSchemaFn( + SpannerConfig spannerConfig, + PCollectionView tx, + PCollectionView dialectView) { this.spannerConfig = spannerConfig; this.tx = tx; + this.dialectView = dialectView; } @Setup @@ -73,13 +85,14 @@ public void teardown() throws Exception { @ProcessElement public void processElement(ProcessContext c) { Transaction transaction = c.sideInput(tx); + Dialect dialect = c.sideInput(dialectView); BatchTransactionId transactionId = transaction.transactionId(); BatchClient batchClient = spannerAccessor.getBatchClient(); BatchReadOnlyTransaction context = batchClient.batchReadOnlyTransaction(transactionId); - InformationSchemaScanner scanner = new InformationSchemaScanner(context); + InformationSchemaScanner scanner = new InformationSchemaScanner(context, dialect); Ddl ddl = scanner.scan(); c.output(ddl); } diff --git a/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java b/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java index 84f1fe094d..b7bd788c56 100644 --- a/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java +++ b/src/main/java/com/google/cloud/teleport/spanner/TextImportTransform.java @@ -17,6 +17,7 @@ import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Mutation; +import com.google.cloud.teleport.spanner.ExportProtos.ProtoDialect; import com.google.cloud.teleport.spanner.TextImportProtos.ImportManifest; import com.google.cloud.teleport.spanner.TextImportProtos.ImportManifest.TableManifest; import com.google.cloud.teleport.spanner.common.Type.Code; @@ -57,6 +58,7 @@ import org.apache.beam.sdk.io.gcp.spanner.Transaction; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Keys; import org.apache.beam.sdk.transforms.MapElements; @@ -95,13 +97,19 @@ public PDone expand(PBegin begin) { PCollectionView tx = begin.apply(SpannerIO.createTransaction().withSpannerConfig(spannerConfig)); + PCollectionView dialectView = + begin + .apply("Read Dialect", new ReadDialect(spannerConfig)) + .apply("Dialect As PCollectionView", View.asSingleton()); + PCollection ddl = - begin.apply("Read Information Schema", new ReadInformationSchema(spannerConfig, tx)); + begin.apply( + "Read Information Schema", new ReadInformationSchema(spannerConfig, tx, dialectView)); PCollectionView ddlView = ddl.apply("Cloud Spanner DDL as view", View.asSingleton()); PCollection manifest = - begin.apply("Read manifest file", new ReadImportManifest(importManifest)); + begin.apply("Read manifest file", new ReadImportManifest(importManifest, dialectView)); PCollection> allFiles = manifest.apply("Resolve data files", new ResolveDataFiles(importManifest, ddlView)); @@ -190,7 +198,8 @@ public void processElement(ProcessContext c) { .withCommitDeadline(Duration.standardMinutes(1)) .withMaxCumulativeBackoff(Duration.standardHours(2)) .withMaxNumMutations(10000) - .withGroupingFactor(100)); + .withGroupingFactor(100) + .withDialectView(dialectView)); previousComputation = result.getOutput(); } @@ -303,23 +312,60 @@ public void processElement(ProcessContext c) { static class ReadImportManifest extends PTransform> { private final ValueProvider importManifest; + private PCollectionView dialectView; + + ReadImportManifest(ValueProvider importManifest, PCollectionView dialectView) { + this.importManifest = importManifest; + this.dialectView = dialectView; + } ReadImportManifest(ValueProvider importManifest) { this.importManifest = importManifest; + this.dialectView = null; } @Override public PCollection expand(PBegin input) { - return input - .apply("Read manifest", FileIO.match().filepattern(importManifest)) - .apply( - "Resource id", - MapElements.into(TypeDescriptor.of(ResourceId.class)) - .via((MatchResult.Metadata::resourceId))) - .apply( - "Read manifest json", - MapElements.into(TypeDescriptor.of(ImportManifest.class)) - .via(ReadImportManifest::readManifest)); + if (dialectView == null) { + dialectView = + input + .getPipeline() + .apply("CreateSingleton", Create.of(Dialect.GOOGLE_STANDARD_SQL)) + .apply("Default Dialect As PCollectionView", View.asSingleton()); + } + PCollection manifest = + input + .apply("Read manifest", FileIO.match().filepattern(importManifest)) + .apply( + "Resource id", + MapElements.into(TypeDescriptor.of(ResourceId.class)) + .via((MatchResult.Metadata::resourceId))) + .apply( + "Read manifest json", + MapElements.into(TypeDescriptor.of(ImportManifest.class)) + .via(ReadImportManifest::readManifest)); + manifest.apply( + "Check dialect", + ParDo.of( + new DoFn() { + + @ProcessElement + public void processElement(ProcessContext c) { + ImportManifest proto = c.element(); + Dialect dialect = c.sideInput(dialectView); + ProtoDialect protoDialect = proto.getDialect(); + if (!protoDialect.name().equals(dialect.name())) { + throw new RuntimeException( + String.format( + "Dialect mismatches: Dialect of the database (%s) is different from" + + " the one in exported manifest (%s).", + dialect, protoDialect)); + } + c.output(dialect); + } + }) + .withSideInputs(dialectView)); + return manifest; } private static ImportManifest readManifest(ResourceId fileResource) { @@ -466,7 +512,7 @@ private static void validateManifest(TableManifest tableManifest, Ddl ddl) { if (table.columns().stream().anyMatch(x -> x.isGenerated())) { throw new RuntimeException( String.format( - "DB table %s has one or more generated columns. An explict column list that " + "DB table %s has one or more generated columns. An explicit column list that " + "excludes the generated columns must be provided in the manifest.", table.name())); } diff --git a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java index 63605eb433..e5543646a0 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/SpannerConverters.java @@ -69,6 +69,7 @@ /** Transforms & DoFns & Options for SpannerIO. */ public class SpannerConverters { + private static final Logger LOG = LoggerFactory.getLogger(SpannerConverters.class); private static final Gson GSON = new GsonBuilder().disableHtmlEscaping().create(); diff --git a/src/main/proto/export.proto b/src/main/proto/export.proto index 6a16653210..075fac44cf 100644 --- a/src/main/proto/export.proto +++ b/src/main/proto/export.proto @@ -5,6 +5,11 @@ package com.google.cloud.teleport.spanner; option java_package = "com.google.cloud.teleport.spanner"; option java_outer_classname = "ExportProtos"; +enum ProtoDialect { + GOOGLE_STANDARD_SQL = 0; + POSTGRESQL = 1; +} + // Protobuf schema for the content of the exported table manifest file. message TableManifest { message File { @@ -47,4 +52,6 @@ message Export { repeated DatabaseOption database_options = 2; // The set of change streams that were exported. repeated Table change_streams = 3; + // The dialect of the database. + ProtoDialect dialect = 4; } diff --git a/src/main/proto/text_import.proto b/src/main/proto/text_import.proto index 6ae1c758fd..ac3e429f83 100644 --- a/src/main/proto/text_import.proto +++ b/src/main/proto/text_import.proto @@ -5,6 +5,8 @@ package com.google.cloud.teleport.spanner; option java_package = "com.google.cloud.teleport.spanner"; option java_outer_classname = "TextImportProtos"; +import "export.proto"; + // Protobuf of the import manifest, consisting of a list of TableManifest. message ImportManifest { // The per-table import manifest. @@ -34,4 +36,7 @@ message ImportManifest { } // The TableManifest of the tables to be imported. repeated TableManifest tables = 1; + + // The dialect of the database to be imported. + ProtoDialect dialect = 2; } diff --git a/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java b/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java index 4b65fa8057..5aa78ac59b 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java +++ b/src/test/java/com/google/cloud/teleport/spanner/CompareDatabases.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.spanner; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; @@ -106,8 +107,14 @@ public PCollection> expand(PBegin begin) { PCollectionView tx = begin.apply(SpannerIO.createTransaction().withSpannerConfig(spanConfig)); + PCollectionView dialectView = + begin + .apply("Read Dialect", new ReadDialect(spanConfig)) + .apply("As PCollectionView", View.asSingleton()); + PCollection sourceDdl = - begin.apply("Read Information Schema", new ReadInformationSchema(spanConfig, tx)); + begin.apply( + "Read Information Schema", new ReadInformationSchema(spanConfig, tx, dialectView)); final PCollectionView ddlView = sourceDdl.apply(View.asSingleton()); @@ -146,6 +153,7 @@ public void processElement(ProcessContext c) { key += struct.getLong(pk.name()); break; case STRING: + case PG_NUMERIC: key += struct.getString(pk.name()); break; case BYTES: @@ -162,6 +170,9 @@ public void processElement(ProcessContext c) { case DATE: key += struct.getDate(pk.name()); break; + case NUMERIC: + key += struct.getBigDecimal(pk.name()); + break; default: throw new IllegalArgumentException("Unsupported PK type " + columnType); } diff --git a/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java b/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java index 4f6b5de4e6..e5e0ac8c09 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/CopyDbTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertThat; import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.ReadOnlyTransaction; import com.google.cloud.teleport.spanner.ExportProtos.Export; import com.google.cloud.teleport.spanner.common.Type; @@ -72,8 +73,18 @@ public void teardown() { } private void createAndPopulate(Ddl ddl, int numBatches) throws Exception { - spannerServer.createDatabase(sourceDb, ddl.statements()); - spannerServer.createDatabase(destinationDb, Collections.emptyList()); + switch (ddl.dialect()) { + case GOOGLE_STANDARD_SQL: + spannerServer.createDatabase(sourceDb, ddl.statements()); + spannerServer.createDatabase(destinationDb, Collections.emptyList()); + break; + case POSTGRESQL: + spannerServer.createPgDatabase(sourceDb, ddl.statements()); + spannerServer.createPgDatabase(destinationDb, Collections.emptyList()); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + ddl.dialect()); + } spannerServer.populateRandomData(sourceDb, ddl, numBatches); } @@ -115,6 +126,106 @@ public void allTypesSchema() throws Exception { runTest(); } + @Test + public void allPgTypesSchema() throws Exception { + // spotless:off + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("age") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .end() + .endTable() + .createTable("AllTYPES") + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("bool_field") + .pgBool() + .endColumn() + .column("int_field") + .pgInt8() + .endColumn() + .column("float_field") + .pgFloat8() + .endColumn() + .column("string_field") + .pgText() + .endColumn() + .column("bytes_field") + .pgBytea() + .endColumn() + .column("timestamp_field") + .pgTimestamptz() + .endColumn() + .column("numeric_field") + .pgNumeric() + .endColumn() + .column("date_field") + .pgDate() + .endColumn() + .column("arr_bool_field") + .type(Type.pgArray(Type.pgBool())) + .endColumn() + .column("arr_int_field") + .type(Type.pgArray(Type.pgInt8())) + .endColumn() + .column("arr_float_field") + .type(Type.pgArray(Type.pgFloat8())) + .endColumn() + .column("arr_string_field") + .type(Type.pgArray(Type.pgVarchar())) + .max() + .endColumn() + .column("arr_bytes_field") + .type(Type.pgArray(Type.pgBytea())) + .max() + .endColumn() + .column("arr_timestamp_field") + .type(Type.pgArray(Type.pgTimestamptz())) + .endColumn() + .column("arr_date_field") + .type(Type.pgArray(Type.pgDate())) + .endColumn() + .column("arr_numeric_field") + .type(Type.pgArray(Type.pgNumeric())) + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .asc("id") + .asc("float_field") + .end() + .interleaveInParent("Users") + .onDeleteCascade() + .endTable() + .build(); + // spotless:on + createAndPopulate(ddl, 100); + runTest(Dialect.POSTGRESQL); + } + @Test public void emptyTables() throws Exception { // spotless:off @@ -170,6 +281,106 @@ public void emptyTables() throws Exception { runTest(); } + @Test + public void emptyPgTables() throws Exception { + // spotless:off + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name").pgVarchar().size(5).endColumn() + .column("age") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .end() + .endTable() + .createTable("AllTYPES") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name").pgVarchar().size(5).endColumn() + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("bool_field") + .pgBool() + .endColumn() + .column("int_field") + .pgInt8() + .endColumn() + .column("float_field") + .pgFloat8() + .endColumn() + .column("string_field") + .pgText() + .endColumn() + .column("bytes_field") + .pgBytea() + .endColumn() + .column("timestamp_field") + .pgTimestamptz() + .endColumn() + .column("numeric_field") + .pgNumeric() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .asc("id") + .end() + .interleaveInParent("Users") + .onDeleteCascade() + .endTable() + .build(); + createAndPopulate(ddl, 10); + + // Add empty tables. + Ddl emptyTables = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("empty_one") + .column("first") + .pgVarchar() + .max() + .endColumn() + .column("second").pgVarchar().size(5).endColumn() + .column("value") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first") + .asc("second") + .end() + .endTable() + .createTable("empty_two") + .column("first") + .pgVarchar() + .max() + .endColumn() + .column("second").pgVarchar().size(5).endColumn() + .column("value") + .pgInt8() + .endColumn() + .column("another_value") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first") + .end() + .endTable() + .build(); + // spotless:on + spannerServer.updateDatabase(sourceDb, emptyTables.createTableStatements()); + runTest(Dialect.POSTGRESQL); + } + @Test public void allEmptyTables() throws Exception { // spotless:off @@ -207,6 +418,76 @@ public void allEmptyTables() throws Exception { runTest(); } + @Test + public void allEmptyPgTables() throws Exception { + // spotless:off + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("age") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .end() + .endTable() + .createTable("AllTYPES") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("bool_field") + .pgBool() + .endColumn() + .column("int_field") + .pgInt8() + .endColumn() + .column("float_field") + .pgFloat8() + .endColumn() + .column("string_field") + .pgText() + .endColumn() + .column("bytes_field") + .pgBytea() + .endColumn() + .column("timestamp_field") + .pgTimestamptz() + .endColumn() + .column("numeric_field") + .pgNumeric() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .asc("id") + .end() + .interleaveInParent("Users") + .onDeleteCascade() + .endTable() + .build(); + // spotless:on + createAndPopulate(ddl, 0); + runTest(Dialect.POSTGRESQL); + } + @Test public void databaseOptions() throws Exception { Ddl.Builder ddlBuilder = Ddl.builder(); @@ -252,7 +533,7 @@ public void databaseOptions() throws Exception { Ddl ddl = ddlBuilder.build(); createAndPopulate(ddl, 100); runTest(); - Ddl destinationDdl = readDdl(destinationDb); + Ddl destinationDdl = readDdl(destinationDb, Dialect.GOOGLE_STANDARD_SQL); List destDbOptions = destinationDdl.setOptionsStatements(destinationDb); assertThat(destDbOptions.size(), is(1)); assertThat( @@ -263,6 +544,83 @@ public void databaseOptions() throws Exception { + "` SET OPTIONS ( version_retention_period = \"6d\" )")); } + @Test + public void pgDatabaseOptions() throws Exception { + Ddl.Builder ddlBuilder = Ddl.builder(Dialect.POSTGRESQL); + // Table Content + // spotless:off + ddlBuilder + .createTable("Users") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name").pgVarchar().size(5).endColumn() + .column("age") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .end() + .endTable() + .createTable("EmploymentData") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name").pgVarchar().size(5).endColumn() + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("age") + .pgInt8() + .endColumn() + .column("address") + .pgVarchar() + .max() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .asc("id") + .end() + .interleaveInParent("Users") + .onDeleteCascade() + .endTable(); + // spotless:on + // Allowed and well-formed database option + List dbOptionList = new ArrayList<>(); + dbOptionList.add( + Export.DatabaseOption.newBuilder() + .setOptionName("version_retention_period") + .setOptionValue("'6d'") + .build()); + // Disallowed database option + dbOptionList.add( + Export.DatabaseOption.newBuilder() + .setOptionName("optimizer_version") + .setOptionValue("1") + .build()); + // Misformed database option + dbOptionList.add( + Export.DatabaseOption.newBuilder() + .setOptionName("123version") + .setOptionValue("xyz") + .build()); + ddlBuilder.mergeDatabaseOptions(dbOptionList); + Ddl ddl = ddlBuilder.build(); + createAndPopulate(ddl, 100); + runTest(Dialect.POSTGRESQL); + Ddl destinationDdl = readDdl(destinationDb, Dialect.POSTGRESQL); + List destDbOptions = destinationDdl.setOptionsStatements(destinationDb); + assertThat(destDbOptions.size(), is(1)); + assertThat( + destDbOptions.get(0), + is("ALTER DATABASE \"" + destinationDb + "\" SET spanner.version_retention_period = '6d'")); + } + @Test public void emptyDb() throws Exception { Ddl ddl = Ddl.builder().build(); @@ -270,6 +628,13 @@ public void emptyDb() throws Exception { runTest(); } + @Test + public void emptyPgDb() throws Exception { + Ddl ddl = Ddl.builder(Dialect.POSTGRESQL).build(); + createAndPopulate(ddl, 0); + runTest(Dialect.POSTGRESQL); + } + @Test public void foreignKeys() throws Exception { // spotless:off @@ -300,6 +665,58 @@ public void foreignKeys() throws Exception { runTest(); } + @Test + public void pgForeignKeys() throws Exception { + // spotless:off + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Ref") + .column("id1") + .pgInt8() + .endColumn() + .column("id2") + .pgInt8() + .endColumn() + .primaryKey() + .asc("id1") + .asc("id2") + .end() + .endTable() + .createTable("Child") + .column("id1") + .pgInt8() + .endColumn() + .column("id2") + .pgInt8() + .endColumn() + .column("id3") + .pgInt8() + .endColumn() + .primaryKey() + .asc("id1") + .asc("id2") + .asc("id3") + .end() + .interleaveInParent("Ref") + // Add some foreign keys that are guaranteed to be satisfied due to interleaving + .foreignKeys( + ImmutableList.of( + "ALTER TABLE \"Child\" ADD CONSTRAINT \"fk1\" FOREIGN KEY (\"id1\") REFERENCES" + + " \"Ref\" (\"id1\")", + "ALTER TABLE \"Child\" ADD CONSTRAINT \"fk2\" FOREIGN KEY (\"id2\") REFERENCES" + + " \"Ref\" (\"id2\")", + "ALTER TABLE \"Child\" ADD CONSTRAINT \"fk3\" FOREIGN KEY (\"id2\") REFERENCES" + + " \"Ref\" (\"id2\")", + "ALTER TABLE \"Child\" ADD CONSTRAINT \"fk4\" FOREIGN KEY (\"id2\", \"id1\") " + + "REFERENCES \"Ref\" (\"id2\", \"id1\")")) + .endTable() + .build(); + // spotless:on + + createAndPopulate(ddl, 100); + runTest(Dialect.POSTGRESQL); + } + // TODO: enable this test once CHECK constraints are enabled // @Test public void checkConstraints() throws Exception { @@ -318,6 +735,32 @@ public void checkConstraints() throws Exception { runTest(); } + @Test + public void pgCheckConstraints() throws Exception { + // spotless:off + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("T") + .column("id") + .pgInt8() + .endColumn() + .column("A") + .pgInt8() + .endColumn() + .primaryKey() + .asc("id") + .end() + .checkConstraints( + ImmutableList.of( + "CONSTRAINT \"ck\" CHECK(LENGTH(CAST(\"A\" AS VARCHAR)) >= '0'::bigint)")) + .endTable() + .build(); + // spotless:on + + createAndPopulate(ddl, 100); + runTest(Dialect.POSTGRESQL); + } + @Test public void changeStreams() throws Exception { Ddl ddl = @@ -364,6 +807,14 @@ public void randomSchema() throws Exception { runTest(); } + @Test + public void randomPgSchema() throws Exception { + Ddl ddl = RandomDdlGenerator.builder(Dialect.POSTGRESQL).setMaxViews(2).build().generate(); + System.out.println(ddl.prettyPrint()); + createAndPopulate(ddl, 100); + runTest(Dialect.POSTGRESQL); + } + @Test public void randomSchemaNoData() throws Exception { Ddl ddl = RandomDdlGenerator.builder().build().generate(); @@ -371,7 +822,18 @@ public void randomSchemaNoData() throws Exception { runTest(); } + @Test + public void randomPgSchemaNoData() throws Exception { + Ddl ddl = RandomDdlGenerator.builder(Dialect.POSTGRESQL).setMaxViews(2).build().generate(); + createAndPopulate(ddl, 0); + runTest(Dialect.POSTGRESQL); + } + private void runTest() { + runTest(Dialect.GOOGLE_STANDARD_SQL); + } + + private void runTest(Dialect dialect) { String tmpDirPath = tmpDir.getRoot().getAbsolutePath(); ValueProvider.StaticValueProvider destination = ValueProvider.StaticValueProvider.of(tmpDirPath); @@ -409,18 +871,18 @@ private void runTest() { PipelineResult compareResult = comparePipeline.run(); compareResult.waitUntilFinish(); - Ddl sourceDdl = readDdl(sourceDb); - Ddl destinationDdl = readDdl(destinationDb); + Ddl sourceDdl = readDdl(sourceDb, dialect); + Ddl destinationDdl = readDdl(destinationDb, dialect); assertThat(sourceDdl.prettyPrint(), equalToCompressingWhiteSpace(destinationDdl.prettyPrint())); } /* Returns the Ddl representing a Spanner database for given a String for the database name */ - private Ddl readDdl(String db) { + private Ddl readDdl(String db, Dialect dialect) { DatabaseClient dbClient = spannerServer.getDbClient(db); Ddl ddl; try (ReadOnlyTransaction ctx = dbClient.readOnlyTransaction()) { - ddl = new InformationSchemaScanner(ctx).scan(); + ddl = new InformationSchemaScanner(ctx, dialect).scan(); } return ddl; } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java index 744798dae4..42cadbf1c7 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportTimestampTest.java @@ -19,9 +19,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.ReadOnlyTransaction; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerOptions; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.InformationSchemaScanner; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.junit.After; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -50,7 +50,7 @@ @Category(IntegrationTest.class) public class ExportTimestampTest { - static String tmpDir = Files.createTempDir().getAbsolutePath(); + private final String tmpDir = Files.createTempDir().getAbsolutePath(); private final Timestamp timestamp = new Timestamp(System.currentTimeMillis()); private final long numericTime = timestamp.getTime(); @@ -80,6 +80,17 @@ public class ExportTimestampTest { @Rule public final SpannerServerResource spannerServer = new SpannerServerResource(); + @Before + public void setup() { + spannerServer.dropDatabase(sourceDb); + spannerServer.dropDatabase(destDbPrefix + chkpt1); + spannerServer.dropDatabase(destDbPrefix + chkpt2); + spannerServer.dropDatabase(destDbPrefix + chkpt3); + spannerServer.dropDatabase(destDbPrefix + chkPt1WithTs); + spannerServer.dropDatabase(destDbPrefix + chkPt2WithTs); + spannerServer.dropDatabase(destDbPrefix + chkPt3WithTs); + } + @After public void teardown() { spannerServer.dropDatabase(sourceDb); @@ -92,7 +103,16 @@ public void teardown() { } private void createAndPopulate(String db, Ddl ddl, int numBatches) throws Exception { - spannerServer.createDatabase(db, ddl.statements()); + switch (ddl.dialect()) { + case GOOGLE_STANDARD_SQL: + spannerServer.createDatabase(db, ddl.statements()); + break; + case POSTGRESQL: + spannerServer.createPgDatabase(db, ddl.statements()); + break; + default: + throw new IllegalArgumentException("Unrecognized dialect: " + ddl.dialect()); + } spannerServer.populateRandomData(db, ddl, numBatches); } @@ -258,6 +278,142 @@ public void runExportWithTsTest() throws Exception { compareDbs(destDbPrefix + chkpt3, destDbPrefix + chkPt3WithTs, comparePipeline3); } + @Test + public void runPgExportWithTsTest() throws Exception { + Ddl ddl = + Ddl.builder(Dialect.POSTGRESQL) + .createTable("Users") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("age") + .pgInt8() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .end() + .endTable() + .createTable("AllTYPES") + .column("first_name") + .pgVarchar() + .max() + .endColumn() + .column("last_name") + .pgVarchar() + .size(5) + .endColumn() + .column("id") + .pgInt8() + .notNull() + .endColumn() + .column("bool_field") + .pgBool() + .endColumn() + .column("int_field") + .pgInt8() + .endColumn() + .column("float_field") + .pgFloat8() + .endColumn() + .column("string_field") + .pgText() + .endColumn() + .column("bytes_field") + .pgBytea() + .endColumn() + .column("timestamp_field") + .pgTimestamptz() + .endColumn() + .column("numeric_field") + .pgNumeric() + .endColumn() + .column("date_field") + .pgDate() + .endColumn() + .primaryKey() + .asc("first_name") + .asc("last_name") + .asc("id") + .end() + .interleaveInParent("Users") + .onDeleteCascade() + .endTable() + .build(); + + // Create initial table and populate + createAndPopulate(sourceDb, ddl, 100); + + // Export the database and note the timestamp ts1 + spannerServer.createPgDatabase(destDbPrefix + chkpt1, Collections.emptyList()); + exportAndImportDbAtTime( + sourceDb, + destDbPrefix + chkpt1, + chkpt1, + "", /* ts = "" */ + exportPipeline1, + importPipeline1); + String chkPt1Ts = getCurrentTimestamp(); + + Thread.sleep(2000); + + // Sleep for a couple of seconds and note the timestamp ts2 + String chkPt2Ts = getCurrentTimestamp(); + + Thread.sleep(2000); + + // Add more records to the table, export the database and note the timestamp ts3 + spannerServer.populateRandomData(sourceDb, ddl, 100); + spannerServer.createPgDatabase(destDbPrefix + chkpt3, Collections.emptyList()); + exportAndImportDbAtTime( + sourceDb, + destDbPrefix + chkpt3, + chkpt3, + "", /* ts = "" */ + exportPipeline2, + importPipeline2); + String chkPt3Ts = getCurrentTimestamp(); + + // Export timestamp with timestamp ts1 + spannerServer.createPgDatabase(destDbPrefix + chkPt1WithTs, Collections.emptyList()); + exportAndImportDbAtTime( + sourceDb, destDbPrefix + chkPt1WithTs, + chkPt1WithTs, chkPt1Ts, + exportPipeline3, importPipeline3); + + // Export timestamp with timestamp ts2 + spannerServer.createPgDatabase(destDbPrefix + chkPt2WithTs, Collections.emptyList()); + exportAndImportDbAtTime( + sourceDb, destDbPrefix + chkPt2WithTs, + chkPt2WithTs, chkPt2Ts, + exportPipeline4, importPipeline4); + + // Export timestamp with timestamp ts3 + spannerServer.createPgDatabase(destDbPrefix + chkPt3WithTs, Collections.emptyList()); + exportAndImportDbAtTime( + sourceDb, + destDbPrefix + chkPt3WithTs, + chkPt3WithTs, + chkPt3Ts, + exportPipeline5, + importPipeline5); + + // Compare databases exported at ts1 and exported later specifying timestamp ts1 + compareDbs( + destDbPrefix + chkpt1, destDbPrefix + chkPt1WithTs, comparePipeline1, Dialect.POSTGRESQL); + // Compare databases exported at ts1 and exported later specifying timestamp ts2 + compareDbs( + destDbPrefix + chkpt1, destDbPrefix + chkPt2WithTs, comparePipeline2, Dialect.POSTGRESQL); + // Compare databases exported at ts3 and exported later specifying timestamp ts3 + compareDbs( + destDbPrefix + chkpt3, destDbPrefix + chkPt3WithTs, comparePipeline3, Dialect.POSTGRESQL); + } + private void exportAndImportDbAtTime( String sourceDb, String destDb, @@ -309,6 +465,11 @@ private void exportAndImportDbAtTime( } private void compareDbs(String sourceDb, String destDb, TestPipeline comparePipeline) { + compareDbs(sourceDb, destDb, comparePipeline, Dialect.GOOGLE_STANDARD_SQL); + } + + private void compareDbs( + String sourceDb, String destDb, TestPipeline comparePipeline, Dialect dialect) { SpannerConfig sourceConfig = spannerServer.getSpannerConfig(sourceDb); SpannerConfig copyConfig = spannerServer.getSpannerConfig(destDb); PCollection mismatchCount = @@ -322,18 +483,16 @@ private void compareDbs(String sourceDb, String destDb, TestPipeline comparePipe PipelineResult compareResult = comparePipeline.run(); compareResult.waitUntilFinish(); - Ddl sourceDdl = readDdl(sourceDb); - Ddl destinationDdl = readDdl(destDb); + Ddl sourceDdl = readDdl(sourceDb, dialect); + Ddl destinationDdl = readDdl(destDb, dialect); assertThat(sourceDdl.prettyPrint(), equalToCompressingWhiteSpace(destinationDdl.prettyPrint())); } - private Ddl readDdl(String db) { - SpannerOptions spannerOptions = SpannerOptions.newBuilder().build(); - Spanner client = spannerOptions.getService(); + private Ddl readDdl(String db, Dialect dialect) { Ddl ddl; try (ReadOnlyTransaction ctx = spannerServer.getDbClient(db).readOnlyTransaction()) { - ddl = new InformationSchemaScanner(ctx).scan(); + ddl = new InformationSchemaScanner(ctx, dialect).scan(); } return ddl; } diff --git a/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java b/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java index 00ff8051d3..4fba32c856 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java @@ -22,9 +22,11 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.TimestampBound; import com.google.cloud.teleport.spanner.ExportProtos.Export; import com.google.cloud.teleport.spanner.ExportProtos.Export.Builder; +import com.google.cloud.teleport.spanner.ExportProtos.ProtoDialect; import com.google.cloud.teleport.spanner.ExportProtos.TableManifest; import com.google.cloud.teleport.spanner.ExportTransform.BuildTableManifests; import com.google.cloud.teleport.spanner.ExportTransform.CombineTableMetadata; @@ -124,10 +126,15 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { ddlBuilder.createChangeStream("changeStream").endChangeStream(); Ddl ddl = ddlBuilder.build(); PCollectionView ddlView = pipeline.apply(Create.of(ddl)).apply(View.asSingleton()); + PCollectionView dialectView = + pipeline + .apply("CreateSingleton", Create.of(Dialect.GOOGLE_STANDARD_SQL)) + .apply("As PCollectionView", View.asSingleton()); PCollection databaseManifest = metadataTables.apply( "Test adding database option to manifest", - ParDo.of(new CreateDatabaseManifest(ddlView)).withSideInputs(ddlView)); + ParDo.of(new CreateDatabaseManifest(ddlView, dialectView)) + .withSideInputs(ddlView, dialectView)); // The output JSON may contain the tables in any order, so a string comparison is not // sufficient. Have to convert the manifest string to a protobuf. Also for the checker function @@ -144,6 +151,7 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { } Export manifestProto = builder1.build(); assertThat(manifestProto.getTablesCount(), is(2)); + assertThat(manifestProto.getDialect(), is(ProtoDialect.GOOGLE_STANDARD_SQL)); String table1Name = manifestProto.getTables(0).getName(); assertThat(table1Name, startsWith("table")); assertThat( diff --git a/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java b/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java index 3a8e3f1542..9fee2c6a24 100644 --- a/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java +++ b/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java @@ -19,18 +19,25 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; +import com.google.cloud.spanner.Dialect; import com.google.cloud.spanner.ReadOnlyTransaction; +import com.google.cloud.teleport.spanner.ExportProtos.ProtoDialect; +import com.google.cloud.teleport.spanner.common.NumericUtils; import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.InformationSchemaScanner; import com.google.protobuf.util.JsonFormat; import java.io.File; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; @@ -114,6 +121,50 @@ public void booleans() throws Exception { .build())); } + @Test + public void pgBooleans() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("booleans"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Boolean columns. + .optionalBoolean("optional_boolean") + .requiredBoolean("required_boolean") + .optionalString("optional_string_boolean") + .requiredString("required_string_boolean"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_boolean\" boolean," + + "\"required_boolean\" boolean NOT NULL," + + "\"optional_string_boolean\" boolean," + + "\"required_string_boolean\" boolean NOT NULL," + + " PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("required_boolean", true) + .set("optional_boolean", false) + .set("required_string_boolean", "FALSE") + .set("optional_string_boolean", "TRUE") + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("required_boolean", false) + .set("optional_boolean", true) + .set("required_string_boolean", "true") + .set("optional_string_boolean", "f") + .build()), + Dialect.POSTGRESQL); + } + @Test public void integers() throws Exception { SchemaBuilder.RecordBuilder record = SchemaBuilder.record("integers"); @@ -165,6 +216,58 @@ public void integers() throws Exception { .build())); } + @Test + public void pgIntegers() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("integers"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Integer columns. + .optionalInt("optional_int") + .requiredInt("required_int") + .requiredLong("required_long") + .optionalLong("optional_long") + .optionalString("optional_string_long") + .requiredString("required_string_long"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_int\" bigint," + + "\"required_int\" bigint NOT NULL," + + "\"optional_long\" bigint," + + "\"required_long\" bigint NOT NULL," + + "\"optional_string_long\" bigint," + + "\"required_string_long\" bigint NOT NULL," + + " PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_int", 1) + .set("optional_long", 2L) + .set("required_long", 3L) + .set("required_int", 4) + .set("optional_string_long", "1000") + .set("required_string_long", "5000") + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("optional_int", 10) + .set("optional_long", 20L) + .set("required_long", 30L) + .set("required_int", 40) + .set("optional_string_long", "10000") + .set("required_string_long", "50000") + .build()), + Dialect.POSTGRESQL); + } + @Test public void floats() throws Exception { SchemaBuilder.RecordBuilder record = SchemaBuilder.record("floats"); @@ -233,6 +336,75 @@ public void floats() throws Exception { .build())); } + @Test + public void pgFloats() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("floats"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Integer columns. + .optionalInt("optional_int") + .requiredInt("required_int") + .requiredLong("required_long") + .optionalLong("optional_long") + // Floating columns + .optionalFloat("optional_float") + .requiredFloat("required_float") + .requiredDouble("required_double") + .optionalDouble("optional_double") + .optionalString("optional_string_double") + .requiredString("required_string_double"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_int\" double precision," + + "\"required_int\" double precision NOT NULL," + + "\"optional_long\" double precision," + + "\"required_long\" double precision NOT NULL," + + "\"optional_float\" double precision," + + "\"required_float\" double precision NOT NULL," + + "\"optional_double\" double precision," + + "\"required_double\" double precision NOT NULL," + + "\"optional_string_double\" double precision," + + "\"required_string_double\" double precision NOT NULL," + + " PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_int", 1) + .set("required_int", 4) + .set("optional_long", 2L) + .set("required_long", 3L) + .set("optional_float", 2.3f) + .set("required_float", 3.4f) + .set("optional_double", 2.5) + .set("required_double", 3.6) + .set("optional_string_double", "100.30") + .set("required_string_double", "0.1e-3") + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("optional_int", 10) + .set("required_int", 40) + .set("optional_long", 20L) + .set("required_long", 30L) + .set("optional_float", 2.03f) + .set("required_float", 3.14f) + .set("optional_double", 2.05) + .set("required_double", 3.16) + .set("optional_string_double", "100.301") + .set("required_string_double", "1.1e-3") + .build()), + Dialect.POSTGRESQL); + } + @Test public void strings() throws Exception { SchemaBuilder.RecordBuilder record = SchemaBuilder.record("strings"); @@ -302,6 +474,76 @@ public void strings() throws Exception { .build())); } + @Test + public void pgStrings() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("strings"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Integer columns. + .optionalInt("optional_int") + .requiredInt("required_int") + .requiredLong("required_long") + .optionalLong("optional_long") + // Floating columns + .optionalFloat("optional_float") + .requiredFloat("required_float") + .requiredDouble("required_double") + .optionalDouble("optional_double") + // String columns + .optionalString("optional_string") + .requiredString("required_string"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_int\" character varying(10)," + + "\"required_int\" character varying NOT NULL," + + "\"optional_long\" character varying," + + "\"required_long\" character varying NOT NULL," + + "\"optional_float\" character varying," + + "\"required_float\" character varying NOT NULL," + + "\"optional_double\" character varying," + + "\"required_double\" character varying NOT NULL," + + "\"optional_string\" character varying," + + "\"required_string\" character varying(30) NOT NULL," + + " PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_int", 1) + .set("required_int", 4) + .set("optional_long", 2L) + .set("required_long", 3L) + .set("optional_float", 2.3f) + .set("required_float", 3.4f) + .set("optional_double", 2.5) + .set("required_double", 3.6) + .set("optional_string", "ONE STRING") + .set("required_string", "TWO STRING") + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("optional_int", 10) + .set("required_int", 40) + .set("optional_long", 20L) + .set("required_long", 30L) + .set("optional_float", 2.03f) + .set("required_float", 3.14f) + .set("optional_double", 2.05) + .set("required_double", 3.16) + .set("optional_string", null) + .set("required_string", "THE STRING") + .build()), + Dialect.POSTGRESQL); + } + @Test public void timestamps() throws Exception { SchemaBuilder.RecordBuilder record = SchemaBuilder.record("timestamps"); @@ -353,6 +595,58 @@ public void timestamps() throws Exception { .build())); } + @Test + public void pgTimestamptzs() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("timestamps"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Long columns. + .requiredLong("required_long") + .optionalLong("optional_long") + // String columns + .optionalString("optional_string") + .requiredString("required_string"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_long\" timestamp with time zone," + + "\"required_long\" timestamp with time zone NOT NULL," + + "\"optional_string\" timestamp with time zone," + + "\"required_string\" timestamp with time zone NOT NULL," + + " PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_long", 5000000L) + .set("required_long", 6000000L) + .set("optional_string", "2018-06-06T21:00:35.312000000Z") + .set("required_string", "2018-06-06T21:00:35.312000000Z") + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("optional_long", 500000330L) + .set("required_long", 6000020000L) + .set("optional_string", "2017-06-06T21:00:35.312000000Z") + .set("required_string", "2017-06-06T21:00:35.312000000Z") + .build(), + new GenericRecordBuilder(schema) + .set("id", 3L) + .set("optional_long", null) + .set("required_long", 6000020000L) + .set("optional_string", null) + .set("required_string", "0001-01-01T00:00:00Z") + .build()), + Dialect.POSTGRESQL); + } + @Test public void dates() throws Exception { // Unfortunately Avro SchemaBuilder has a limitation of not allowing nullable LogicalTypes. @@ -395,8 +689,166 @@ public void dates() throws Exception { .build())); } - // TODO: enable this test once generated columns are supported. - // @Test + @Test + public void pgDates() throws Exception { + // Unfortunately Avro SchemaBuilder has a limitation of not allowing nullable LogicalTypes. + Schema dateType = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + Schema schema = + SchemaBuilder.record("dates") + .fields() + // Primary key. + .requiredLong("id") + // String columns + .optionalString("optional_string") + .requiredString("required_string") + .name("required_int") + .type(dateType) + .noDefault() + .endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_string\" date," + + "\"required_string\" date NOT NULL," + + "\"required_int\" date NOT NULL," + + "PRIMARY KEY (\"id\")) "; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_string", "2018-03-04") + .set("required_string", "2018-04-04") + .set("required_int", 2) + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("optional_string", null) + .set("required_string", "2018-01-02") + .set("required_int", 3) + .build()), + Dialect.POSTGRESQL); + } + + @Test + public void pgNumerics() throws Exception { + // Unfortunately Avro SchemaBuilder has a limitation of not allowing nullable LogicalTypes. + Schema numericType = + LogicalTypes.decimal(NumericUtils.PG_MAX_PRECISION, NumericUtils.PG_MAX_SCALE) + .addToSchema(Schema.create(Schema.Type.BYTES)); + Schema schema = + SchemaBuilder.record("numerics") + .fields() + // Primary key. + .requiredLong("id") + // Bytes columns + .optionalBytes("optional_bytes") + .requiredBytes("required_bytes") + .name("required_decimal") + .type(numericType) + .noDefault() + .endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_bytes\" numeric," + + "\"required_bytes\" numeric NOT NULL," + + "\"required_decimal\" numeric NOT NULL," + + "PRIMARY KEY (\"id\")) "; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_bytes", ByteBuffer.wrap(NumericUtils.pgStringToBytes("NaN"))) + .set( + "required_bytes", + ByteBuffer.wrap(NumericUtils.pgStringToBytes("81313218843.0000000"))) + .set( + "required_decimal", + ByteBuffer.wrap(NumericUtils.pgStringToBytes("10.0000000010"))) + .build(), + new GenericRecordBuilder(schema) + .set("id", 2L) + .set("optional_bytes", null) + .set( + "required_bytes", + ByteBuffer.wrap(NumericUtils.pgStringToBytes("-0.218311815433216841313811548"))) + .set( + "required_decimal", + ByteBuffer.wrap(NumericUtils.pgStringToBytes("-711384543184123843132.483124"))) + .build()), + Dialect.POSTGRESQL); + } + + @Test + public void pgNumericArray() throws Exception { + // Unfortunately Avro SchemaBuilder has a limitation of not allowing nullable LogicalTypes. + Schema numericType = + LogicalTypes.decimal(NumericUtils.PG_MAX_PRECISION, NumericUtils.PG_MAX_SCALE) + .addToSchema(Schema.create(Schema.Type.BYTES)); + Schema numericArrayType = + SchemaBuilder.builder().array().items().type(wrapAsNullable(numericType)); + + Schema schema = + SchemaBuilder.record("numerics") + .fields() + // Primary key. + .requiredLong("id") + .name("required_numeric_arr") + .type(numericArrayType) + .noDefault() + .endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"required_numeric_arr\" numeric[] NOT NULL," + + "PRIMARY KEY (\"id\")) "; + + StringBuilder maxPgNumeric = new StringBuilder(); + StringBuilder minPgNumeric = new StringBuilder("-"); + for (int i = 0; i < NumericUtils.PG_MAX_PRECISION - NumericUtils.PG_MAX_SCALE; i++) { + maxPgNumeric.append("9"); + minPgNumeric.append("9"); + } + maxPgNumeric.append("."); + minPgNumeric.append("."); + for (int i = 0; i < NumericUtils.PG_MAX_SCALE; i++) { + maxPgNumeric.append("9"); + minPgNumeric.append("9"); + } + + String[] arrValues = { + null, + "NaN", + null, + maxPgNumeric.toString(), + minPgNumeric.toString(), + "-9305028.140032", + "-25398514232141142.012479", + "1999999999.1246" + }; + List pgNumericArr = + Stream.of(arrValues) + .map(x -> x == null ? null : ByteBuffer.wrap(NumericUtils.pgStringToBytes(x))) + .collect(Collectors.toList()); + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("required_numeric_arr", pgNumericArr) + .build()), + Dialect.POSTGRESQL); + } + + @Test public void generatedColumns() throws Exception { SchemaBuilder.RecordBuilder record = SchemaBuilder.record("generatedColumns"); SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); @@ -426,6 +878,72 @@ public void generatedColumns() throws Exception { .build())); } + @Test + public void pgGeneratedColumns() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("generatedColumns"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Integer columns. + .optionalLong("optional_generated") + .requiredLong("required_generated") + .optionalLong("optional_generated_2"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_generated\" bigint GENERATED ALWAYS AS (\"id\") STORED," + + "\"required_generated\" bigint NOT NULL GENERATED ALWAYS AS (\"id\") STORED," + + "\"optional_generated_2\" bigint NOT NULL GENERATED ALWAYS AS (\"id\") STORED," + + "PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_generated", 1L) + .set("required_generated", 1L) + .build()), + Dialect.POSTGRESQL); + } + + @Test + public void pgDefaultColumns() throws Exception { + SchemaBuilder.RecordBuilder record = SchemaBuilder.record("defaultColumns"); + SchemaBuilder.FieldAssembler fieldAssembler = record.fields(); + + fieldAssembler + // Primary key. + .requiredLong("id") + // Integer columns. + .optionalLong("optional_default") + .requiredLong("required_default") + .optionalLong("optional_default_2"); + Schema schema = fieldAssembler.endRecord(); + String spannerSchema = + "CREATE TABLE \"AvroTable\" (" + + "\"id\" bigint NOT NULL," + + "\"optional_default\" bigint DEFAULT '1'::bigint," + + "\"required_default\" bigint DEFAULT '2'::bigint," + + "\"optional_default_2\" bigint DEFAULT '3'::bigint," + + "PRIMARY KEY (\"id\"))"; + + runTest( + schema, + spannerSchema, + Arrays.asList( + new GenericRecordBuilder(schema) + .set("id", 1L) + .set("optional_default", 1L) + .set("required_default", 1L) + .build()), + Dialect.POSTGRESQL); + } + @Test public void changeStreams() throws Exception { Map avroFiles = new HashMap<>(); @@ -536,6 +1054,12 @@ public void changeStreams() throws Exception { private void runTest(Schema avroSchema, String spannerSchema, Iterable records) throws Exception { + runTest(avroSchema, spannerSchema, records, Dialect.GOOGLE_STANDARD_SQL); + } + + private void runTest( + Schema avroSchema, String spannerSchema, Iterable records, Dialect dialect) + throws Exception { // Create the Avro file to be imported. String fileName = "avroFile.avro"; ExportProtos.Export exportProto = @@ -548,8 +1072,9 @@ private void runTest(Schema avroSchema, String spannerSchema, Iterable overrides) { List numerics = value.isNull() ? null : value.getNumericArray(); builder.set(columnName).toNumericArray(numerics); break; + case PG_NUMERIC: + List pgNumerics = value.isNull() ? null : value.getStringArray(); + builder.set(columnName).toPgNumericArray(pgNumerics); + break; } break; default: From 2d9ee80cb48396cf58acc28c529ee44f23f7fb28 Mon Sep 17 00:00:00 2001 From: pranavbhandari Date: Thu, 12 May 2022 12:12:52 -0700 Subject: [PATCH 137/145] Fix exceptions thrown in incorrect location. PiperOrigin-RevId: 448307572 --- .../teleport/templates/common/BigQueryConverters.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/google/cloud/teleport/templates/common/BigQueryConverters.java b/src/main/java/com/google/cloud/teleport/templates/common/BigQueryConverters.java index 1a64ec700a..a715671441 100644 --- a/src/main/java/com/google/cloud/teleport/templates/common/BigQueryConverters.java +++ b/src/main/java/com/google/cloud/teleport/templates/common/BigQueryConverters.java @@ -157,17 +157,20 @@ public PCollectionTuple expand(PCollection> failsafeE public void processElement(ProcessContext context) { FailsafeElement element = context.element(); String json = element.getPayload(); - + TableRow row; try { - TableRow row = convertJsonToTableRow(json); - context.output(row); + row = convertJsonToTableRow(json); } catch (Exception e) { context.output( failureTag(), FailsafeElement.of(element) .setErrorMessage(e.getMessage()) .setStacktrace(Throwables.getStackTraceAsString(e))); + return; } + // The call to ouput should be outside of the try/catch block to prevent + // catching errors from downstream transforms. + context.output(row); } }) .withOutputTags(successTag(), TupleTagList.of(failureTag()))); From 2382eb8d519125183f3dfbb6ac9d2f200e0dcc2f Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Thu, 12 May 2022 15:15:48 -0700 Subject: [PATCH 138/145] Update Spanner to GCS template to include Spanner metadata table parameter PiperOrigin-RevId: 448350312 --- v2/googlecloud-to-googlecloud/pom.xml | 1 - .../SpannerChangeStreamsToGcsOptions.java | 16 ++- .../templates/SpannerChangeStreamsToGcs.java | 12 +- .../SpannerChangeStreamsToGcsTest.java | 110 +++++++++++++++++- 4 files changed, 126 insertions(+), 13 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/pom.xml b/v2/googlecloud-to-googlecloud/pom.xml index dd6fe6ff0c..63a72af2ab 100644 --- a/v2/googlecloud-to-googlecloud/pom.xml +++ b/v2/googlecloud-to-googlecloud/pom.xml @@ -77,7 +77,6 @@ - org.apache.beam diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java index c9408968e7..db58a6071b 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToGcsOptions.java @@ -49,9 +49,9 @@ public interface SpannerChangeStreamsToGcsOptions @Description("The Spanner database to read from.") @Validation.Required - String getSpannerDatabaseId(); + String getSpannerDatabase(); - void setSpannerDatabaseId(String spannerDatabaseId); + void setSpannerDatabase(String spannerDatabase); @Description("The Spanner instance to use for the change stream metadata table.") @Validation.Required @@ -61,9 +61,17 @@ public interface SpannerChangeStreamsToGcsOptions @Description("The Spanner database to use for the change stream metadata table.") @Validation.Required - String getSpannerMetadataDatabaseId(); + String getSpannerMetadataDatabase(); - void setSpannerMetadataDatabaseId(String spannerMetadataDatabaseId); + void setSpannerMetadataDatabase(String spannerMetadataDatabase); + + @Description( + "The Cloud Spanner change streams Connector metadata table name to use. If not provided, a" + + " Cloud Spanner change streams Connector metadata table will automatically be created" + + " during the pipeline flow.") + String getSpannerMetadataTableName(); + + void setSpannerMetadataTableName(String value); @Description("The Spanner change stream to read from.") @Validation.Required diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java index 0653b047f8..bd9075fb45 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcs.java @@ -70,9 +70,9 @@ public static PipelineResult run(SpannerChangeStreamsToGcsOptions options) { // Get the Spanner project, instance, database, and change stream parameters. String projectId = getProjectId(options); String instanceId = options.getSpannerInstanceId(); - String databaseId = options.getSpannerDatabaseId(); + String databaseId = options.getSpannerDatabase(); String metadataInstanceId = options.getSpannerMetadataInstanceId(); - String metadataDatabaseId = options.getSpannerMetadataDatabaseId(); + String metadataDatabaseId = options.getSpannerMetadataDatabase(); String changeStreamName = options.getSpannerChangeStreamName(); // Retrieve and parse the start / end timestamps. @@ -103,6 +103,11 @@ public static PipelineResult run(SpannerChangeStreamsToGcsOptions options) { } options.setExperiments(experiments); + String metadataTableName = + options.getSpannerMetadataTableName() == null + ? null + : options.getSpannerMetadataTableName(); + final RpcPriority rpcPriority = options.getRpcPriority(); pipeline .apply( @@ -118,7 +123,8 @@ public static PipelineResult run(SpannerChangeStreamsToGcsOptions options) { .withChangeStreamName(changeStreamName) .withInclusiveStartAt(startTimestamp) .withInclusiveEndAt(endTimestamp) - .withRpcPriority(rpcPriority)) + .withRpcPriority(rpcPriority) + .withMetadataTable(metadataTableName)) .apply( "Creating " + options.getWindowDuration() + " Window", Window.into(FixedWindows.of(DurationUtils.parseDuration(options.getWindowDuration())))) diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java index 4503718f2d..493113cfcc 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsTest.java @@ -28,13 +28,16 @@ import com.google.cloud.teleport.v2.transforms.FileFormatFactorySpannerChangeStreams; import com.google.cloud.teleport.v2.utils.DurationUtils; import com.google.cloud.teleport.v2.utils.WriteToGCSUtility.FileFormat; +import com.google.gson.Gson; import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -68,6 +71,7 @@ public final class SpannerChangeStreamsToGcsTest { @Rule public final transient TestPipeline pipeline = TestPipeline.create(); private static final String AVRO_FILENAME_PREFIX = "avro-output-"; + private static final String TEXT_FILENAME_PREFIX = "text-output-"; private static final Integer NUM_SHARDS = 1; private static final String TEST_PROJECT = "span-cloud-testing"; private static final String TEST_INSTANCE = "changestream"; @@ -115,6 +119,32 @@ public Void apply(Iterable actual } } + @SuppressWarnings("DefaultAnnotationParam") + private static class VerifyDataChangeRecordText + implements SerializableFunction, Void> { + @Override + public Void apply(Iterable actualIter) { + // Make sure actual is the right length, and is a + // subset of expected. + List actual = new ArrayList<>(); + for (String dataChangeRecordString : actualIter) { + DataChangeRecord s = new Gson().fromJson(dataChangeRecordString, DataChangeRecord.class); + actual.add(s); + assertEquals(TEST_TABLE, s.getTableName()); + assertTrue(s.getCommitTimestamp().getSeconds() > 0); + assertTrue(s.getPartitionToken() != null && s.getPartitionToken().length() > 0); + assertTrue(s.getServerTransactionId() != null && s.getServerTransactionId().length() > 0); + assertTrue(s.getRecordSequence() != null && s.getRecordSequence().length() > 0); + assertTrue(!s.getRowType().isEmpty()); + assertTrue(!s.getMods().isEmpty()); + assertTrue(s.getNumberOfRecordsInTransaction() > 0); + assertTrue(s.getNumberOfPartitionsInTransaction() > 0); + assertTrue(s.getMetadata() != null); + } + return null; + } + } + private String generateDatabaseName() { return TEST_DATABASE_PREFIX + "_" @@ -217,9 +247,6 @@ public void testInvalidWindowDuration() { @Category(IntegrationTest.class) // This test can only be run locally with the following command: // mvn -Dexcluded.spanner.tests="" -Dtest=SpannerChangeStreamsToGcsTest test - // TODO(nancyxu): Add an integration test for writing into GCS text when the connector can be - // run in parallel testing. Should happen after this PR is submitted: - // https://github.com/apache/beam/pull/17036 public void testWriteToGCSAvro() throws Exception { // Create a test database. String testDatabase = generateDatabaseName(); @@ -259,9 +286,9 @@ public void testWriteToGCSAvro() throws Exception { PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); options.setSpannerProjectId(TEST_PROJECT); options.setSpannerInstanceId(TEST_INSTANCE); - options.setSpannerDatabaseId(testDatabase); + options.setSpannerDatabase(testDatabase); options.setSpannerMetadataInstanceId(TEST_INSTANCE); - options.setSpannerMetadataDatabaseId(testDatabase); + options.setSpannerMetadataDatabase(testDatabase); options.setSpannerChangeStreamName(TEST_CHANGE_STREAM); options.setStartTimestamp(startTimestamp.toString()); @@ -291,4 +318,77 @@ public void testWriteToGCSAvro() throws Exception { // Drop the database. spannerServer.dropDatabase(testDatabase); } + + @Test + @Category(IntegrationTest.class) + // This test can only be run locally with the following command: + // mvn -Dexcluded.spanner.tests="" -Dtest=SpannerChangeStreamsToGcsTest test + public void testWriteToGCSText() throws Exception { + // Create a test database. + String testDatabase = generateDatabaseName(); + fakeDir = tmpDir.newFolder("output").getAbsolutePath(); + fakeTempLocation = tmpDir.newFolder("temporaryLocation").getAbsolutePath(); + + spannerServer.dropDatabase(testDatabase); + + // Create a table. + List statements = new ArrayList(); + final String createTable = + "CREATE TABLE " + + TEST_TABLE + + " (" + + "user_id INT64 NOT NULL," + + "name STRING(MAX) " + + ") PRIMARY KEY(user_id)"; + final String createChangeStream = "CREATE CHANGE STREAM " + TEST_CHANGE_STREAM + " FOR Users"; + statements.add(createTable); + statements.add(createChangeStream); + spannerServer.createDatabase(testDatabase, statements); + + Timestamp startTimestamp = Timestamp.now(); + + // Create a mutation for the table that will generate 1 data change record. + List mutations = new ArrayList<>(); + mutations.add( + Mutation.newInsertBuilder(TEST_TABLE).set("user_id").to(1).set("name").to("Name1").build()); + mutations.add( + Mutation.newInsertBuilder(TEST_TABLE).set("user_id").to(2).set("name").to("Name2").build()); + + spannerServer.getDbClient(testDatabase).write(mutations); + + Timestamp endTimestamp = Timestamp.now(); + + SpannerChangeStreamsToGcsOptions options = + PipelineOptionsFactory.create().as(SpannerChangeStreamsToGcsOptions.class); + options.setSpannerProjectId(TEST_PROJECT); + options.setSpannerInstanceId(TEST_INSTANCE); + options.setSpannerDatabase(testDatabase); + options.setSpannerMetadataInstanceId(TEST_INSTANCE); + options.setSpannerMetadataDatabase(testDatabase); + options.setSpannerChangeStreamName(TEST_CHANGE_STREAM); + + options.setStartTimestamp(startTimestamp.toString()); + options.setEndTimestamp(endTimestamp.toString()); + List experiments = new ArrayList(); + options.setExperiments(experiments); + + options.setOutputFileFormat(FileFormat.TEXT); + options.setGcsOutputDirectory(fakeDir); + options.setOutputFilenamePrefix(TEXT_FILENAME_PREFIX); + options.setNumShards(NUM_SHARDS); + options.setTempLocation(fakeTempLocation); + + // Run the pipeline. + PipelineResult result = run(options); + result.waitUntilFinish(); + + // Read from the output Avro file to assert that 1 data change record has been generated. + PCollection dataChangeRecords = + pipeline.apply("readRecords", TextIO.read().from(fakeDir + "/text-output-*.txt")); + PAssert.that(dataChangeRecords).satisfies(new VerifyDataChangeRecordText()); + pipeline.run(); + + // Drop the database. + spannerServer.dropDatabase(testDatabase); + } } From 07b0e921eea8069691ea74ac81e66f5b1ada3e2b Mon Sep 17 00:00:00 2001 From: dhercher Date: Sun, 15 May 2022 01:22:06 -0700 Subject: [PATCH 139/145] DatastreamToSQL: clean empty or empty string numeric data to null PiperOrigin-RevId: 448769626 --- .../v2/utils/DatastreamToPostgresDML.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java index f8cd88f56e..e0bd47336b 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java @@ -82,8 +82,22 @@ public String cleanDataTypeValueSql( return columnValue; } switch (dataType.toUpperCase()) { + case "INT2": + case "INT4": + case "INT8": + case "FLOAT4": + case "FLOAT8": + case "SMALLINT": + case "INTEGER": + case "BIGINT": case "DECIMAL": - if (columnValue.equals("")) { + case "NUMERIC": + case "REAL": + case "DOUBLE PRECISION": + case "SMALLSERIAL": + case "SERIAL": + case "BIGSERIAL": + if (columnValue.equals("") || columnValue.equals("''")) { return getNullValueSql(); } break; From 16db6f16c69f7221bb60980cefef676576c8dbe1 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Wed, 18 May 2022 12:56:12 -0700 Subject: [PATCH 140/145] Fix naming of Spanner change stream to BigQuery Dataflow template PiperOrigin-RevId: 449554470 --- ...SpannerChangeStreamsToBigQueryOptions.java | 22 +++++++++---------- .../SpannerChangeStreamsToBigQuery.java | 4 ++-- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java index 8d58517b89..f17bbbb17a 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java @@ -35,29 +35,29 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setSpannerProjectId(String projectId); - @Description("The Spanner instance ID that contains the Change Stream.") + @Description("The Spanner instance ID that contains the change stream.") @Validation.Required String getSpannerInstanceId(); void setSpannerInstanceId(String value); - @Description("The Spanner database ID that contains the Change Stream.") + @Description("The Spanner database that contains the change stream.") @Validation.Required - String getSpannerDatabaseId(); + String getSpannerDatabase(); - void setSpannerDatabaseId(String value); + void setSpannerDatabase(String value); - @Description("The Spanner metadata instance ID that's used by the Change Stream connector.") + @Description("The Spanner metadata instance ID that's used by the change stream connector.") @Validation.Required String getSpannerMetadataInstanceId(); void setSpannerMetadataInstanceId(String value); - @Description("The Spanner metadata database ID that's used by the Change Stream connector.") + @Description("The Spanner metadata database that's used by the change stream connector.") @Validation.Required - String getSpannerMetadataDatabaseId(); + String getSpannerMetadataDatabase(); - void setSpannerMetadataDatabaseId(String value); + void setSpannerMetadataDatabase(String value); @Description( "The Cloud Spanner change streams Connector metadata table name to use. If not provided, a" @@ -67,7 +67,7 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setSpannerMetadataTableName(String value); - @Description("The name of the Spanner Change Stream.") + @Description("The name of the Spanner change stream.") @Validation.Required String getSpannerChangeStreamName(); @@ -88,7 +88,7 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setSpannerHost(String value); @Description( - "The starting DateTime to use for reading Change Streams" + "The starting DateTime to use for reading change streams" + " (https://tools.ietf.org/html/rfc3339). Defaults to pipeline start time.") @Default.String("") String getStartTimestamp(); @@ -96,7 +96,7 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setStartTimestamp(String startTimestamp); @Description( - "The ending DateTime to use for reading Change Streams" + "The ending DateTime to use for reading change streams" + " (https://tools.ietf.org/html/rfc3339). The default value is \"max\", which represents" + " an infinite time in the future.") @Default.String("") diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java index af8a8e1c42..d789f1ea78 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java @@ -169,14 +169,14 @@ public static PipelineResult run(SpannerChangeStreamsToBigQueryOptions options) .withHost(ValueProvider.StaticValueProvider.of(options.getSpannerHost())) .withProjectId(spannerProjectId) .withInstanceId(options.getSpannerInstanceId()) - .withDatabaseId(options.getSpannerDatabaseId()) + .withDatabaseId(options.getSpannerDatabase()) .withRpcPriority(options.getSpannerRpcPriority()); SpannerIO.ReadChangeStream readChangeStream = SpannerIO.readChangeStream() .withSpannerConfig(spannerConfig) .withMetadataInstance(options.getSpannerMetadataInstanceId()) - .withMetadataDatabase(options.getSpannerMetadataDatabaseId()) + .withMetadataDatabase(options.getSpannerMetadataDatabase()) .withChangeStreamName(options.getSpannerChangeStreamName()) .withInclusiveStartAt(startTimestamp) .withInclusiveEndAt(endTimestamp) From e70192fa0dda7321533e2c95bf9ac51b4473c634 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Thu, 19 May 2022 13:13:30 -0700 Subject: [PATCH 141/145] Upgrade google-cloud-bigquery from 1.128.0 to 2.9.0, and fix JSON support for Dataflow Spanner change stream to BigQuery template PiperOrigin-RevId: 449817379 --- v2/common/pom.xml | 2 +- .../schemautils/SpannerToBigQueryUtils.java | 4 +- .../SchemaUtilsTest.java | 110 +++++++++++++----- 3 files changed, 82 insertions(+), 34 deletions(-) diff --git a/v2/common/pom.xml b/v2/common/pom.xml index 31e19b535a..5fcba65630 100644 --- a/v2/common/pom.xml +++ b/v2/common/pom.xml @@ -29,7 +29,7 @@ 1.8.2 1.15.0 - 1.128.0 + 2.9.0 1.8 1.9 3.15.8 diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java index 40cb356afa..7113c37f5d 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerToBigQueryUtils.java @@ -61,7 +61,7 @@ private static TableFieldSchema spannerColumnToBigQueryIOField( } else if (spannerType.equals(Type.array(Type.int64()))) { bigQueryField.setType("INT64"); } else if (spannerType.equals(Type.array(Type.json()))) { - bigQueryField.setType("STRING"); + bigQueryField.setType("JSON"); } else if (spannerType.equals(Type.array(Type.numeric()))) { bigQueryField.setType("NUMERIC"); } else if (spannerType.equals(Type.array(Type.string()))) { @@ -93,7 +93,7 @@ private static TableFieldSchema spannerColumnToBigQueryIOField( bigQueryType = StandardSQLTypeName.INT64; break; case JSON: - bigQueryType = StandardSQLTypeName.STRING; + bigQueryType = StandardSQLTypeName.JSON; break; case NUMERIC: bigQueryType = StandardSQLTypeName.NUMERIC; diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java index 4377cda8d4..17a79a2a83 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java @@ -54,7 +54,9 @@ import static com.google.common.truth.Truth.assertThat; import static org.mockito.Mockito.when; +import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.Field; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.Key; import com.google.cloud.spanner.ReadContext; @@ -304,37 +306,83 @@ public void testAppendToSpannerKey() { @Test public void testSpannerColumnsToBigQueryIOFields() { - String bigQueryIOFieldsStr = - SpannerToBigQueryUtils.spannerColumnsToBigQueryIOFields(spannerColumnsOfAllTypes) - .toString(); - // Remove redundant information. - bigQueryIOFieldsStr = - bigQueryIOFieldsStr.replace( - "classInfo=[categories, collationSpec, description, fields, maxLength, mode, name," - + " policyTags, precision, scale, type], ", - ""); - bigQueryIOFieldsStr = bigQueryIOFieldsStr.replace("GenericData", ""); - - assertThat(bigQueryIOFieldsStr) - .isEqualTo( - "[{{mode=NULLABLE, name=BooleanCol, type=BOOL}}, " - + "{{mode=NULLABLE, name=BytesCol, type=BYTES}}, " - + "{{mode=NULLABLE, name=DateCol, type=DATE}}, " - + "{{mode=NULLABLE, name=Float64Col, type=FLOAT64}}, " - + "{{mode=NULLABLE, name=Int64Col, type=INT64}}, " - + "{{mode=NULLABLE, name=JsonCol, type=STRING}}, " - + "{{mode=NULLABLE, name=NumericCol, type=NUMERIC}}, " - + "{{mode=NULLABLE, name=StringCol, type=STRING}}, " - + "{{mode=NULLABLE, name=TimestampCol, type=TIMESTAMP}}, " - + "{{mode=REPEATED, name=BooleanArrayCol, type=BOOL}}, " - + "{{mode=REPEATED, name=BytesArrayCol, type=BYTES}}, " - + "{{mode=REPEATED, name=DateArrayCol, type=DATE}}, " - + "{{mode=REPEATED, name=Float64ArrayCol, type=FLOAT64}}, " - + "{{mode=REPEATED, name=Int64ArrayCol, type=INT64}}, " - + "{{mode=REPEATED, name=JsonArrayCol, type=STRING}}, " - + "{{mode=REPEATED, name=NumericArrayCol, type=NUMERIC}}, " - + "{{mode=REPEATED, name=StringArrayCol, type=STRING}}, " - + "{{mode=REPEATED, name=TimestampArrayCol, type=TIMESTAMP}}]"); + List tableFields = + ImmutableList.of( + new TableFieldSchema() + .setName(BOOLEAN_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("BOOL"), + new TableFieldSchema() + .setName(BYTES_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("BYTES"), + new TableFieldSchema() + .setName(DATE_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("DATE"), + new TableFieldSchema() + .setName(FLOAT64_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("FLOAT64"), + new TableFieldSchema() + .setName(INT64_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("INT64"), + new TableFieldSchema() + .setName(JSON_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("JSON"), + new TableFieldSchema() + .setName(NUMERIC_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("NUMERIC"), + new TableFieldSchema() + .setName(STRING_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("STRING"), + new TableFieldSchema() + .setName(TIMESTAMP_COL) + .setMode(Field.Mode.NULLABLE.name()) + .setType("TIMESTAMP"), + new TableFieldSchema() + .setName(BOOLEAN_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("BOOL"), + new TableFieldSchema() + .setName(BYTES_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("BYTES"), + new TableFieldSchema() + .setName(DATE_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("DATE"), + new TableFieldSchema() + .setName(FLOAT64_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("FLOAT64"), + new TableFieldSchema() + .setName(INT64_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("INT64"), + new TableFieldSchema() + .setName(JSON_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("JSON"), + new TableFieldSchema() + .setName(NUMERIC_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("NUMERIC"), + new TableFieldSchema() + .setName(STRING_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("STRING"), + new TableFieldSchema() + .setName(TIMESTAMP_ARRAY_COL) + .setMode(Field.Mode.REPEATED.name()) + .setType("TIMESTAMP")); + + assertThat(SpannerToBigQueryUtils.spannerColumnsToBigQueryIOFields(spannerColumnsOfAllTypes)) + .isEqualTo(tableFields); } @Test From 901278b958bfc0b154ca6f0a6ec34d6150ce7a78 Mon Sep 17 00:00:00 2001 From: Cloud Teleport Date: Fri, 20 May 2022 13:21:21 -0700 Subject: [PATCH 142/145] Two fixes for Spanner change stream to BigQuery template. PiperOrigin-RevId: 450050043 --- ...SpannerChangeStreamsToBigQueryOptions.java | 9 ++++---- .../BigQueryDynamicDestinations.java | 21 ++++++++++++++++++- .../FailsafeModJsonToTableRowTransformer.java | 16 ++++++-------- .../SpannerChangeStreamsToBigQuery.java | 9 +++++++- .../schemautils/SpannerUtils.java | 15 ++++++++----- ...lsafeModJsonToTableRowTransformerTest.java | 3 ++- .../SchemaUtilsTest.java | 2 +- 7 files changed, 52 insertions(+), 23 deletions(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java index f17bbbb17a..9bf18810be 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerChangeStreamsToBigQueryOptions.java @@ -134,11 +134,12 @@ public interface SpannerChangeStreamsToBigQueryOptions extends DataflowPipelineO void setDlqRetryMinutes(Integer value); + // TODO(haikuo-google): Test this in UIF test. @Description( - "Comma separated list of fields to be ignored, these could be fields of tracked tables, or" - + " metadata fields which are _metadata_spanner_mod_type, _metadata_spanner_table_name," - + " _metadata_spanner_commit_timestamp, _metadata_spanner_server_transaction_id," - + " _metadata_spanner_record_sequence," + "Comma separated list of fields (case sensitive) to be ignored, these could be fields of" + + " tracked tables, or metadata fields which are _metadata_spanner_mod_type," + + " _metadata_spanner_table_name, _metadata_spanner_commit_timestamp," + + " _metadata_spanner_server_transaction_id, _metadata_spanner_record_sequence," + " _metadata_spanner_is_last_record_in_transaction_in_partition," + " _metadata_spanner_number_of_records_in_transaction," + " _metadata_spanner_number_of_partitions_in_transaction," diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java index 515a245848..3512196587 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/BigQueryDynamicDestinations.java @@ -27,7 +27,9 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; import com.google.cloud.teleport.v2.transforms.BigQueryConverters; +import com.google.common.collect.ImmutableSet; import java.io.Serializable; +import java.util.ArrayList; import java.util.List; import java.util.Map; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations; @@ -46,6 +48,7 @@ public final class BigQueryDynamicDestinations private final Map spannerTableByName; private final String bigQueryProject, bigQueryDataset, bigQueryTableTemplate; + private final ImmutableSet ignoreFields; public static BigQueryDynamicDestinations of( BigQueryDynamicDestinationsOptions bigQueryDynamicDestinationsOptions) { @@ -65,6 +68,7 @@ private BigQueryDynamicDestinations( BigQueryDynamicDestinationsOptions bigQueryDynamicDestinationsOptions, Map spannerTableByName) { this.spannerTableByName = spannerTableByName; + this.ignoreFields = bigQueryDynamicDestinationsOptions.getIgnoreFields(); this.bigQueryProject = bigQueryDynamicDestinationsOptions.getBigQueryProject(); this.bigQueryDataset = bigQueryDynamicDestinationsOptions.getBigQueryDataset(); this.bigQueryTableTemplate = bigQueryDynamicDestinationsOptions.getBigQueryTableTemplate(); @@ -98,7 +102,18 @@ public TableSchema getSchema(KV destination) { String spannerTableName = (String) tableRow.get(BigQueryUtils.BQ_CHANGELOG_FIELD_NAME_TABLE_NAME); TrackedSpannerTable spannerTable = spannerTableByName.get(spannerTableName); + List fields = getFields(spannerTable); + List filteredFields = new ArrayList<>(); + for (TableFieldSchema field : fields) { + if (!ignoreFields.contains(field.getName())) { + filteredFields.add(field); + } + } + + return new TableSchema().setFields(filteredFields); + } + private List getFields(TrackedSpannerTable spannerTable) { List fields = SpannerToBigQueryUtils.spannerColumnsToBigQueryIOFields(spannerTable.getAllColumns()); @@ -151,7 +166,7 @@ public TableSchema getSchema(KV destination) { .setType(StandardSQLTypeName.TIMESTAMP.name()) .setMode(requiredMode)); - return new TableSchema().setFields(fields); + return fields; } /** @@ -164,6 +179,8 @@ public abstract static class BigQueryDynamicDestinationsOptions implements Seria public abstract String getChangeStreamName(); + public abstract ImmutableSet getIgnoreFields(); + public abstract String getBigQueryProject(); public abstract String getBigQueryDataset(); @@ -180,6 +197,8 @@ abstract static class Builder { abstract Builder setChangeStreamName(String changeStreamName); + abstract Builder setIgnoreFields(ImmutableSet ignoreFields); + abstract Builder setBigQueryProject(String bigQueryProject); abstract Builder setBigQueryDataset(String bigQueryDataset); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java index 738e7ecb34..6bd7cce064 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformer.java @@ -34,16 +34,15 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerToBigQueryUtils; import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.SpannerUtils; import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.common.collect.ImmutableSet; import io.grpc.CallOptions; import io.grpc.Context; import io.grpc.MethodDescriptor; import java.io.IOException; import java.io.ObjectInputStream; import java.io.Serializable; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; @@ -119,7 +118,7 @@ public static class FailsafeModJsonToTableRowFn private final SpannerConfig spannerConfig; private final String spannerChangeStream; private Map spannerTableByName; - private final Set ignoreFields; + private final ImmutableSet ignoreFields; public TupleTag transformOut; public TupleTag> transformDeadLetterOut; private transient CallContextConfigurator callContextConfigurator; @@ -127,17 +126,14 @@ public static class FailsafeModJsonToTableRowFn public FailsafeModJsonToTableRowFn( SpannerConfig spannerConfig, String spannerChangeStream, - String ignoreFieldsStr, + ImmutableSet ignoreFields, TupleTag transformOut, TupleTag> transformDeadLetterOut) { this.spannerConfig = spannerConfig; this.spannerChangeStream = spannerChangeStream; this.transformOut = transformOut; this.transformDeadLetterOut = transformDeadLetterOut; - this.ignoreFields = new HashSet<>(); - for (String ignoreField : ignoreFieldsStr.split(",")) { - ignoreFields.add(ignoreField); - } + this.ignoreFields = ignoreFields; } private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { @@ -342,7 +338,7 @@ public abstract static class FailsafeModJsonToTableRowOptions implements Seriali public abstract String getSpannerChangeStream(); - public abstract String getIgnoreFields(); + public abstract ImmutableSet getIgnoreFields(); public abstract FailsafeElementCoder getCoder(); @@ -357,7 +353,7 @@ abstract static class Builder { abstract Builder setSpannerChangeStream(String spannerChangeStream); - abstract Builder setIgnoreFields(String ignoreFields); + abstract Builder setIgnoreFields(ImmutableSet ignoreFields); abstract Builder setCoder(FailsafeElementCoder coder); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java index d789f1ea78..0c3fdf07ee 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SpannerChangeStreamsToBigQuery.java @@ -25,6 +25,7 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; import com.google.cloud.teleport.v2.transforms.DLQWriteTransform; import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.common.collect.ImmutableSet; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -219,12 +220,17 @@ public void process( .and(retryableDlqFailsafeModJson) .apply("Merge Source And DLQ Mod JSON", Flatten.pCollections()); + ImmutableSet.Builder ignoreFieldsBuilder = ImmutableSet.builder(); + for (String ignoreField : options.getIgnoreFields().split(",")) { + ignoreFieldsBuilder.add(ignoreField); + } + ImmutableSet ignoreFields = ignoreFieldsBuilder.build(); FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions failsafeModJsonToTableRowOptions = FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRowOptions.builder() .setSpannerConfig(spannerConfig) .setSpannerChangeStream(options.getSpannerChangeStreamName()) - .setIgnoreFields(options.getIgnoreFields()) + .setIgnoreFields(ignoreFields) .setCoder(FAILSAFE_ELEMENT_CODER) .build(); FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow failsafeModJsonToTableRow = @@ -239,6 +245,7 @@ public void process( BigQueryDynamicDestinations.BigQueryDynamicDestinationsOptions.builder() .setSpannerConfig(spannerConfig) .setChangeStreamName(options.getSpannerChangeStreamName()) + .setIgnoreFields(ignoreFields) .setBigQueryProject(getBigQueryProjectId(options)) .setBigQueryDataset(options.getBigQueryDataset()) .setBigQueryTableTemplate(options.getBigQueryChangelogTableNameTemplate()) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java index 92e9d03a88..1d863f692f 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/schemautils/SpannerUtils.java @@ -74,11 +74,13 @@ public Map getSpannerTableByName() { private Map getSpannerTableByName( Set spannerTableNames, Map> spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName) { - Map> spannerColumnsByTableName = - getSpannerColumnsByTableName( - spannerTableNames, spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName); Map> keyColumnNameByTableName = getKeyColumnNameByTableName(spannerTableNames); + Map> spannerColumnsByTableName = + getSpannerColumnsByTableName( + spannerTableNames, + keyColumnNameByTableName, + spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName); Map result = new HashMap<>(); for (String tableName : spannerColumnsByTableName.keySet()) { @@ -104,6 +106,7 @@ private Map getSpannerTableByName( */ private Map> getSpannerColumnsByTableName( Set spannerTableNames, + Map> keyColumnNameByTableName, Map> spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName) { Map> result = new HashMap<>(); StringBuilder sqlStringBuilder = @@ -127,11 +130,13 @@ private Map> getSpannerColumnsByTableName( String tableName = columnsResultSet.getString(INFORMATION_SCHEMA_TABLE_NAME); String columnName = columnsResultSet.getString(INFORMATION_SCHEMA_COLUMN_NAME); // Skip if the columns of the table is tracked explicitly, and the specified column is not - // tracked. + // tracked. Primary key columns are always tracked. if (spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName.containsKey(tableName) && !spannerColumnNamesExplicitlyTrackedByChangeStreamByTableName .get(tableName) - .contains(columnName)) { + .contains(columnName) + && (!keyColumnNameByTableName.containsKey(tableName) + || !keyColumnNameByTableName.get(tableName).contains(columnName))) { continue; } diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java index 7f1a6eaf71..5eab4663ac 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/FailsafeModJsonToTableRowTransformerTest.java @@ -96,6 +96,7 @@ import com.google.cloud.teleport.v2.templates.spannerchangestreamstobigquery.schemautils.BigQueryUtils; import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.Pipeline; @@ -355,7 +356,7 @@ private static FailsafeModJsonToTableRow getFailsafeModJsonToTableRow( .setSpannerConfig(SPANNER_SERVER.getSpannerConfig(spannerDatabaseName)) .setSpannerChangeStream(TEST_SPANNER_CHANGE_STREAM) .setCoder(SpannerChangeStreamsToBigQuery.FAILSAFE_ELEMENT_CODER) - .setIgnoreFields("") + .setIgnoreFields(ImmutableSet.of()) .build(); return new FailsafeModJsonToTableRowTransformer.FailsafeModJsonToTableRow( failsafeModJsonToTableRowOptions); diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java index 17a79a2a83..b5d50d2eef 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/spannerchangestreamstobigquery/SchemaUtilsTest.java @@ -184,7 +184,7 @@ public void testChangeStreamTrackOneTable() { } @Test - public void testChangeStreamTrackTwoColumns() { + public void testChangeStreamTrackOneColumn() { mockInformationSchemaChangeStreamsQuery(false); mockInformationSchemaChangeStreamTablesQuery(); mockInformationSchemaColumnsQuery(); From a0f84bcad9de2483a96f85334708a24d96269707 Mon Sep 17 00:00:00 2001 From: Zachary Houfek Date: Tue, 24 May 2022 11:42:19 -0400 Subject: [PATCH 143/145] Remove hive-to-bigquery template --- v2/hive-to-bigquery/README.md | 204 ------------------ v2/hive-to-bigquery/pom.xml | 180 ---------------- .../teleport/v2/templates/HiveToBigQuery.java | 179 --------------- .../teleport/v2/templates/package-info.java | 18 -- .../src/main/resources/core-site.xml | 44 ---- .../hive-to-bigquery-command-spec.json | 7 - v2/pom.xml | 1 - 7 files changed, 633 deletions(-) delete mode 100644 v2/hive-to-bigquery/README.md delete mode 100644 v2/hive-to-bigquery/pom.xml delete mode 100644 v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/HiveToBigQuery.java delete mode 100644 v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java delete mode 100644 v2/hive-to-bigquery/src/main/resources/core-site.xml delete mode 100644 v2/hive-to-bigquery/src/main/resources/hive-to-bigquery-command-spec.json diff --git a/v2/hive-to-bigquery/README.md b/v2/hive-to-bigquery/README.md deleted file mode 100644 index d4f28b7ea9..0000000000 --- a/v2/hive-to-bigquery/README.md +++ /dev/null @@ -1,204 +0,0 @@ -# Hive to BigQuery Dataflow Template - -The [HiveToBigQuery](src/main/java/com/google/cloud/teleport/v2/templates -/HiveToBigQuery.java) pipeline is a -batch pipeline which ingests data from Hive and outputs - the resulting records to BigQuery. - -### Requirements -* Java 8 -* Maven -* Hive 2 -* Docker -* The Hive input table exists. -* The Hive metastore URI and HDFS URI are reachable from the Dataflow worker - machines. - -## Getting Started -### Building Template -This is a Flex template meaning that the pipeline code will be containerized and the container will be used to launch the Dataflow pipeline. - -#### Compiling the pipeline -Execute the following command from the directory containing the parent pom.xml (v2/): - -```shell script -mvn clean compile -pl hive-to-bigquery -am -``` - -#### Execuing unit tests -Execute the following command from the directory containing the parent pom.xml (v2/): - -```shell script -mvn clean test -pl hive-to-bigquery -am -``` - -### Building Container Image -Execute the following command from the directory containing the parent pom.xml (v2/): -* Set environment variables that will be used in the build process. - -```sh -export PROJECT=my-project -export IMAGE_NAME=hive-to-bigquery -export BUCKET_NAME=gs:// -export TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} -export BASE_CONTAINER_IMAGE=gcr.io/dataflow-templates-base/java8-template-launcher-base -export BASE_CONTAINER_IMAGE_VERSION=latest -export APP_ROOT=/template/${IMAGE_NAME} -export COMMAND_SPEC=${APP_ROOT}/resources/${IMAGE_NAME}-command-spec.json -export TEMPLATE_IMAGE_SPEC=${BUCKET_NAME}/images/${IMAGE_NAME}-image-spec.json - -gcloud config set project ${PROJECT} -``` -* Build and push image to Google Container Repository -```sh -mvn clean package \ - -Dimage=${TARGET_GCR_IMAGE} \ - -Dbase-container-image=${BASE_CONTAINER_IMAGE} \ - -Dbase-container-image.version=${BASE_CONTAINER_IMAGE_VERSION} \ - -Dapp-root=${APP_ROOT} \ - -Dcommand-spec=${COMMAND_SPEC} \ - -am -pl ${IMAGE_NAME} -``` - -#### Creating Image Spec -Create a file with the metadata required for launching the Flex template. Once created, this file should -be placed in GCS. - -__Note:__ The ```image``` property would point to the ```${TARGER_GCR_IMAGE}``` defined previously. - -```sh -echo '{ - "image": "'${TARGET_GCR_IMAGE}'", - "metadata": { - "name": "Replicates from a Hive table into BigQuery", - "description": "Hive to BigQuery", - "parameters": [ - { - "name": "metastoreUri", - "label": "metastoreUri", - "helpText": "thrift server URI such as thrift://thrift-server-host:port", - "paramType": "TEXT", - "isOptional": false - }, - { - "name": "hiveDatabaseName", - "label": "hiveDatabaseName", - "helpText": "Input Hive Database Name", - "paramType": "TEXT", - "is_optional": false - }, - { - "name": "hiveTableName", - "label": "hiveTableName", - "helpText": "Input Hive table name", - "paramType": "TEXT", - "is_optional": false - }, - { - "name": "outputTableSpec", - "label": "outputTableSpec", - "helpText": "Output BigQuery table spec such as myproject:mydataset.mytable", - "paramType": "TEXT", - "is_optional": false - }, - { - "name": "hivePartitionCols", - "label": "hivePartitionCols", - "helpText": "the name of the columns that are partitions such as [\"col1\", \"col2\"]", - "paramType": "TEXT", - "is_optional": true - }, - { - "name": "filterString", - "label": "filterString", - "helpText": "the filter details", - "paramType": "TEXT", - "is_optional": true - }, - { - "name": "partitionType", - "label": "partitionType", - "helpText": "partition type in BigQuery. Currently, only Time is - available", - "paramType": "TEXT", - "is_optional": true - }, - { - "name": "partitionCol", - "label": "partitionCol", - "helpText": "the name of column that is the partition in BigQuery", - "paramType": "TEXT", - "is_optional": true - }, - { - "name": "maxRetryAttempts", - "label": "Max retry attempts", - "helpText": "Max retry attempts, must be > 0. Default: no retries", - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "maxRetryDuration", - "label": "Max retry duration in milliseconds", - "helpText": "Max retry duration in milliseconds, must be > 0. Default: no retries", - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "autoscalingAlgorithm","label":"Autoscaling algorithm to use", - "helpText": "Autoscaling algorithm to use: THROUGHPUT_BASED", - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "numWorkers","label":"Number of workers Dataflow will start with", - "helpText": "Number of workers Dataflow will start with", - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "maxNumWorkers","label":"Maximum number of workers Dataflow job will use", - "helpText": "Maximum number of workers Dataflow job will use", - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "workerMachineType","label":"Worker Machine Type to use in Dataflow Job", - "helpText": "Machine Type to Use: n1-standard-4", - "paramType": "TEXT", - "isOptional": true - } - ] - }, - "sdkInfo": { - "language": "JAVA" - } -}' > image_spec.json -gsutil cp image_spec.json ${TEMPLATE_IMAGE_SPEC} -rm image_spec.json -``` - -### Executing Template - -The template requires the following parameters: - -* metastoreUri: thrift server URI -* hiveDatabaseName: Input Hive database name -* hiveTableName: Input Hive table name -* outputTableSpec: Output BigQuery table spec - -The templatehas the following optional parameters: -* hivePartitionCols: the name of the columns that are partitions in Hive table -* filterString: the filter details -* partitionType: partition type in BigQuery -* partitionCol: the name of the column that is partition in BigQuery - -Template can be executed using the ```gcloud``` sdk. -__**Note:**__ To use the gcloud command-line tool to run Flex templates, you must have [Cloud SDK](https://cloud.google.com/sdk/downloads) version 284.0.0 or higher. - -```sh -export JOB_NAME="${IMAGE_NAME}-`date +%Y%m%d-%H%M%S-%N`" -gcloud dataflow flex-template run ${JOB_NAME} \ - --template-file-gcs-location=${TEMPLATE_IMAGE_SPEC} \ - --parameters="metastoreUri=thrift://my-ip-address:9083,hiveDatabaseName=myhivedbname,hiveTableName=myhivetable,outputTableSpec=my-project:my_dataset.my_table" -``` diff --git a/v2/hive-to-bigquery/pom.xml b/v2/hive-to-bigquery/pom.xml deleted file mode 100644 index 48a6d6e509..0000000000 --- a/v2/hive-to-bigquery/pom.xml +++ /dev/null @@ -1,180 +0,0 @@ - - - - - dynamic-templates - com.google.cloud.teleport.v2 - 1.0-SNAPSHOT - - hive-to-bigquery - 4.0.0 - - - 2.1.0 - 1.0.1 - - - - - com.google.cloud.teleport.v2 - common - 1.0-SNAPSHOT - compile - - - org.apache.logging.log4j - log4j-1.2-api - 2.17.0 - - - org.apache.logging.log4j - log4j-web - 2.17.0 - - - org.apache.logging.log4j - log4j-slf4j-impl - 2.17.0 - - - org.apache.hive - hive-common - ${hive.version} - - - org.codehaus.jackson - * - - - com.fasterxml.jackson.core - jackson-databind - - - - - org.apache.hive - hive-metastore - ${hive.version} - - - org.codehaus.jackson - * - - - org.apache.thrift - libthrift - - - org.apache.hbase - hbase-client - - - - - org.apache.hive.hcatalog - hive-hcatalog-core - ${hive.version} - - - org.codehaus.jackson - * - - - org.apache.thrift - libthrift - - - commons-httpclient - commons-httpclient - - - org.antlr - antlr-runtime - - - org.apache.hadoop - hadoop-client - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - jdk.tools - jdk.tools - - - - - org.antlr - antlr-runtime - 3.4 - - - org.apache.thrift - libthrift - 0.9.3 - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - org.codehaus.jackson - * - - - org.slf4j - slf4j-log4j12 - - - - - org.apache.beam - beam-sdks-java-io-hadoop-file-system - - - org.apache.beam - beam-sdks-java-io-hcatalog - - - - - ${mvn-target-dir} - - - com.google.cloud.tools - jib-maven-plugin - - - package - - build - - - - - - - diff --git a/v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/HiveToBigQuery.java b/v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/HiveToBigQuery.java deleted file mode 100644 index 8caad2231f..0000000000 --- a/v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/HiveToBigQuery.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Copyright (C) 2020 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.v2.templates; - -import com.google.api.services.bigquery.model.TimePartitioning; -import com.google.cloud.teleport.v2.options.BigQueryCommonOptions.WriteOptions; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -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.bigquery.BigQueryIO; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; -import org.apache.beam.sdk.io.hcatalog.HCatToRow; -import org.apache.beam.sdk.io.hcatalog.HCatalogIO; -import org.apache.beam.sdk.io.hdfs.HadoopFileSystemOptions; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.hadoop.conf.Configuration; - -/** - * The {@link HiveToBigQuery} pipeline is a batch pipeline which ingests data from Hive and outputs - * the resulting records to BigQuery. - */ -public final class HiveToBigQuery { - - /** - * The {@link Options} class provides the custom execution options passed by the executor at the - * command-line. - */ - public interface Options extends DataflowPipelineOptions, HadoopFileSystemOptions, WriteOptions { - @Description("hive metastore uri") - @Required - String getMetastoreUri(); - - void setMetastoreUri(String value); - - @Description("Hive database name") - @Required - String getHiveDatabaseName(); - - void setHiveDatabaseName(String value); - - @Description("Hive table name") - @Required - String getHiveTableName(); - - void setHiveTableName(String value); - - @Description("comma separated list of the columns of Hive partition columns") - List getHivePartitionCols(); - - void setHivePartitionCols(List value); - - @Description("the filter details") - String getFilterString(); - - void setFilterString(String value); - } - - /** - * The main entry-point for pipeline execution. This method will start the pipeline but will not - * wait for it's execution to finish. If blocking execution is required, use the {@link - * HiveToBigQuery#run(Options)} method to start the pipeline and invoke {@code - * result.waitUntilFinish()} on the {@link PipelineResult}. - * - * @param args The command-line args passed by the executor. - */ - public static void main(String[] args) { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - run(options); - } - - /** - * Configures the HCatalog source. - * - * @param options The execution options. - * @param configProperties The configuration properties like metastore URI - * @return hiveRead Configured HCatalogIO.Read method - */ - private static HCatalogIO.Read getHCatalogIORead( - Options options, Map configProperties) { - HCatalogIO.Read hiveRead = - HCatalogIO.read() - .withConfigProperties(configProperties) - .withDatabase(options.getHiveDatabaseName()) - .withTable(options.getHiveTableName()); - - if (options.getHivePartitionCols() != null) { - hiveRead = hiveRead.withPartitionCols(options.getHivePartitionCols()); - } - - if (options.getFilterString() != null) { - hiveRead = hiveRead.withFilter(options.getFilterString()); - } - return hiveRead; - } - - /** - * Configures the HCatalog source. - * - * @param options The execution options. - * @param configProperties The configuration properties like metastore URI - * @return bqWrite Configured BigQueryIO.Write method - */ - private static BigQueryIO.Write getBigQueryIOWrite(Options options, PCollection rows) { - BigQueryIO.Write bqWrite = - BigQueryIO.write() - .to(options.getOutputTableSpec()) - .withSchema(BigQueryUtils.toTableSchema(rows.getSchema())) - .withFormatFunction(BigQueryUtils.toTableRow()) - .withCreateDisposition(CreateDisposition.valueOf(options.getCreateDisposition())) - .withWriteDisposition(WriteDisposition.valueOf(options.getWriteDisposition())) - .withExtendedErrorInfo(); - - if (options.getPartitionType() == "Time") { - bqWrite = - bqWrite.withTimePartitioning(new TimePartitioning().setField(options.getPartitionCol())); - } - return bqWrite; - } - - /** - * 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) { - // Set Hadoop configurations - Configuration hadoopConf = new Configuration(); - options.setHdfsConfiguration(Collections.singletonList(hadoopConf)); - - Map configProperties = new HashMap(); - configProperties.put("hive.metastore.uris", options.getMetastoreUri()); - - // Create the pipeline - Pipeline pipeline = Pipeline.create(options); - - PCollection rows = - pipeline - /* - * Step #1: Read hive table rows from Hive. - */ - .apply( - "Read from Hive source", - HCatToRow.fromSpec(getHCatalogIORead(options, configProperties))); - - /* - * Step #2: Write table rows out to BigQuery - */ - rows.apply("Write records to BigQuery", getBigQueryIOWrite(options, rows)); - return pipeline.run(); - } -} diff --git a/v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java b/v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java deleted file mode 100644 index 166004826e..0000000000 --- a/v2/hive-to-bigquery/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java +++ /dev/null @@ -1,18 +0,0 @@ -/* - * Copyright (C) 2020 Google 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. - */ - -/** Templates for moving Hive records into BigQuery. */ -package com.google.cloud.teleport.v2.templates; diff --git a/v2/hive-to-bigquery/src/main/resources/core-site.xml b/v2/hive-to-bigquery/src/main/resources/core-site.xml deleted file mode 100644 index 0c44b94077..0000000000 --- a/v2/hive-to-bigquery/src/main/resources/core-site.xml +++ /dev/null @@ -1,44 +0,0 @@ - - - - - - - - - - - - fs.defaultFS - hdfs:/// - - - fs.hdfs.impl - org.apache.hadoop.hdfs.DistributedFileSystem - - - fs.file.impl - org.apache.hadoop.fs.LocalFileSystem - - - fs.gs.impl - com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem - - - fs.AbstractFileSystem.gs.impl - com.google.cloud.hadoop.fs.gcs.GoogleHadoopFS - - diff --git a/v2/hive-to-bigquery/src/main/resources/hive-to-bigquery-command-spec.json b/v2/hive-to-bigquery/src/main/resources/hive-to-bigquery-command-spec.json deleted file mode 100644 index e07192bda0..0000000000 --- a/v2/hive-to-bigquery/src/main/resources/hive-to-bigquery-command-spec.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "mainClass": "com.google.cloud.teleport.v2.templates.HiveToBigQuery", - "classPath": "/template/hive-to-bigquery/*:/template/hive-to-bigquery/libs/*:/template/hive-to-bigquery/classes", - "defaultParameterValues": { - "labels": "{\"goog-dataflow-provided-template-type\":\"flex\", \"goog-dataflow-provided-template-name\":\"hive_to_bigquery\"}" - } -} diff --git a/v2/pom.xml b/v2/pom.xml index f80cc1d480..0fbd62f6c0 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -681,7 +681,6 @@ file-format-conversion googlecloud-to-elasticsearch googlecloud-to-googlecloud - hive-to-bigquery kafka-to-bigquery kafka-to-gcs kudu-to-bigquery From 12e7c9e085c735f570e8f454f5a1ba102e5bf5ab Mon Sep 17 00:00:00 2001 From: Andrei Gurau Date: Wed, 25 May 2022 10:25:23 -0400 Subject: [PATCH 144/145] Add GCS to Splunk template spotless --- v2/googlecloud-to-splunk/pom.xml | 84 ++++++ .../teleport/v2/coders/SplunkEventCoder.java | 102 +++++++ .../v2/coders/SplunkWriteErrorCoder.java | 80 +++++ .../teleport/v2/coders/package-info.java | 18 ++ .../teleport/v2/templates/GCSToSplunk.java | 255 ++++++++++++++++ .../teleport/v2/templates/package-info.java | 18 ++ .../v2/transforms/SplunkConverters.java | 266 +++++++++++++++++ .../teleport/v2/transforms/package-info.java | 18 ++ .../resources/gcs-to-splunk-command-spec.json | 7 + .../v2/coders/SplunkEventCoderTest.java | 57 ++++ .../v2/coders/SplunkWriteErrorCoderTest.java | 51 ++++ .../v2/templates/GCSToSplunkTest.java | 276 ++++++++++++++++++ .../v2/transforms/SplunkConvertersTest.java | 247 ++++++++++++++++ .../src/test/resources/no_header.csv | 1 + .../src/test/resources/splunkUdf.js | 30 ++ .../src/test/resources/testSchema.json | 14 + .../src/test/resources/with_headers.csv | 2 + v2/pom.xml | 1 + 18 files changed, 1527 insertions(+) create mode 100644 v2/googlecloud-to-splunk/pom.xml create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkEventCoder.java create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoder.java create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/package-info.java create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/GCSToSplunk.java create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/SplunkConverters.java create mode 100644 v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java create mode 100644 v2/googlecloud-to-splunk/src/main/resources/gcs-to-splunk-command-spec.json create mode 100644 v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkEventCoderTest.java create mode 100644 v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoderTest.java create mode 100644 v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/templates/GCSToSplunkTest.java create mode 100644 v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/transforms/SplunkConvertersTest.java create mode 100644 v2/googlecloud-to-splunk/src/test/resources/no_header.csv create mode 100644 v2/googlecloud-to-splunk/src/test/resources/splunkUdf.js create mode 100644 v2/googlecloud-to-splunk/src/test/resources/testSchema.json create mode 100644 v2/googlecloud-to-splunk/src/test/resources/with_headers.csv diff --git a/v2/googlecloud-to-splunk/pom.xml b/v2/googlecloud-to-splunk/pom.xml new file mode 100644 index 0000000000..9101193b83 --- /dev/null +++ b/v2/googlecloud-to-splunk/pom.xml @@ -0,0 +1,84 @@ + + + + + dynamic-templates + com.google.cloud.teleport.v2 + 1.0-SNAPSHOT + + googlecloud-to-splunk + 4.0.0 + + + 2.38.0 + 4.13.1 + 1.0.1 + + + + + com.google.cloud.teleport.v2 + common + 1.0-SNAPSHOT + compile + + + org.apache.beam + beam-sdks-java-io-splunk + ${beam-splunk.version} + + + + com.google.truth + truth + ${truth.version} + test + + + com.google.guava + guava + + + + + junit + junit + ${junit.version} + test + + + + + ${mvn-target-dir} + + + com.google.cloud.tools + jib-maven-plugin + + + package + + build + + + + + + + \ No newline at end of file diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkEventCoder.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkEventCoder.java new file mode 100644 index 0000000000..991c21faaa --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkEventCoder.java @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2022 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.v2.coders; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.splunk.SplunkEvent; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** A {@link org.apache.beam.sdk.coders.Coder} for {@link SplunkEvent} objects. */ +public final class SplunkEventCoder extends AtomicCoder { + + private static final SplunkEventCoder SPLUNK_EVENT_CODER = new SplunkEventCoder(); + + private static final TypeDescriptor TYPE_DESCRIPTOR = + new TypeDescriptor() {}; + private static final StringUtf8Coder STRING_UTF_8_CODER = StringUtf8Coder.of(); + private static final NullableCoder STRING_NULLABLE_CODER = + NullableCoder.of(STRING_UTF_8_CODER); + private static final NullableCoder LONG_NULLABLE_CODER = + NullableCoder.of(BigEndianLongCoder.of()); + + private static final int VERSION = 1; + + public static SplunkEventCoder of() { + return SPLUNK_EVENT_CODER; + } + + @Override + public void encode(SplunkEvent value, OutputStream out) throws IOException { + out.write(VERSION); + + LONG_NULLABLE_CODER.encode(value.time(), out); + STRING_NULLABLE_CODER.encode(value.host(), out); + STRING_NULLABLE_CODER.encode(value.source(), out); + STRING_NULLABLE_CODER.encode(value.sourceType(), out); + STRING_NULLABLE_CODER.encode(value.index(), out); + STRING_UTF_8_CODER.encode(value.event(), out); + } + + @Override + public SplunkEvent decode(InputStream in) throws IOException { + SplunkEvent.Builder builder = SplunkEvent.newBuilder(); + + int v = in.read(); + + Long time = LONG_NULLABLE_CODER.decode(in); + if (time != null) { + builder.withTime(time); + } + + String host = STRING_NULLABLE_CODER.decode(in); + if (host != null) { + builder.withHost(host); + } + + String source = STRING_NULLABLE_CODER.decode(in); + if (source != null) { + builder.withSource(source); + } + + String sourceType = STRING_NULLABLE_CODER.decode(in); + if (sourceType != null) { + builder.withSourceType(sourceType); + } + + String index = STRING_NULLABLE_CODER.decode(in); + if (index != null) { + builder.withIndex(index); + } + + String event = STRING_UTF_8_CODER.decode(in); + if (event != null) { + builder.withEvent(event); + } + + return builder.create(); + } + + @Override + public TypeDescriptor getEncodedTypeDescriptor() { + return TYPE_DESCRIPTOR; + } +} diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoder.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoder.java new file mode 100644 index 0000000000..843f8ee9b3 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoder.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2022 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.v2.coders; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.splunk.SplunkWriteError; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** A {@link org.apache.beam.sdk.coders.Coder} for {@link SplunkWriteError} objects. */ +public final class SplunkWriteErrorCoder extends AtomicCoder { + + private static final SplunkWriteErrorCoder SPLUNK_WRITE_ERROR_CODER = new SplunkWriteErrorCoder(); + + private static final TypeDescriptor TYPE_DESCRIPTOR = + new TypeDescriptor() {}; + private static final StringUtf8Coder STRING_UTF_8_CODER = StringUtf8Coder.of(); + private static final NullableCoder STRING_NULLABLE_CODER = + NullableCoder.of(STRING_UTF_8_CODER); + private static final NullableCoder INTEGER_NULLABLE_CODER = + NullableCoder.of(BigEndianIntegerCoder.of()); + + public static SplunkWriteErrorCoder of() { + return SPLUNK_WRITE_ERROR_CODER; + } + + @Override + public void encode(SplunkWriteError value, OutputStream out) throws CoderException, IOException { + INTEGER_NULLABLE_CODER.encode(value.statusCode(), out); + STRING_NULLABLE_CODER.encode(value.statusMessage(), out); + STRING_NULLABLE_CODER.encode(value.payload(), out); + } + + @Override + public SplunkWriteError decode(InputStream in) throws CoderException, IOException { + + SplunkWriteError.Builder builder = SplunkWriteError.newBuilder(); + + Integer statusCode = INTEGER_NULLABLE_CODER.decode(in); + if (statusCode != null) { + builder.withStatusCode(statusCode); + } + + String statusMessage = STRING_NULLABLE_CODER.decode(in); + if (statusMessage != null) { + builder.withStatusMessage(statusMessage); + } + + String payload = STRING_NULLABLE_CODER.decode(in); + if (payload != null) { + builder.withPayload(payload); + } + + return builder.create(); + } + + @Override + public TypeDescriptor getEncodedTypeDescriptor() { + return TYPE_DESCRIPTOR; + } +} diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/package-info.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/package-info.java new file mode 100644 index 0000000000..77a7e02f71 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/coders/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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. + */ + +/** Google Cloud to Splunk coders for Google Cloud Teleport. */ +package com.google.cloud.teleport.v2.coders; diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/GCSToSplunk.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/GCSToSplunk.java new file mode 100644 index 0000000000..fe64bc631f --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/GCSToSplunk.java @@ -0,0 +1,255 @@ +/* + * Copyright (C) 2022 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.v2.templates; + +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; +import com.google.cloud.teleport.v2.coders.SplunkEventCoder; +import com.google.cloud.teleport.v2.transforms.CsvConverters; +import com.google.cloud.teleport.v2.transforms.CsvConverters.LineToFailsafeJson; +import com.google.cloud.teleport.v2.transforms.CsvConverters.ReadCsv; +import com.google.cloud.teleport.v2.transforms.ErrorConverters.LogErrors; +import com.google.cloud.teleport.v2.transforms.SplunkConverters; +import com.google.cloud.teleport.v2.transforms.SplunkConverters.FailsafeStringToSplunkEvent; +import com.google.cloud.teleport.v2.transforms.SplunkConverters.SplunkOptions; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.common.annotations.VisibleForTesting; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.splunk.SplunkEvent; +import org.apache.beam.sdk.io.splunk.SplunkIO; +import org.apache.beam.sdk.io.splunk.SplunkWriteError; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link com.google.cloud.teleport.v2.templates.GCSToSplunk} pipeline is a batch pipeline which + * ingests data from GCS, optionally executes a UDF, converts the output to {@link + * org.apache.beam.sdk.io.splunk.SplunkEvent}s and writes those records into Splunk's HEC endpoint. + * Any errors which occur in the execution of the UDF, conversion to {@link + * org.apache.beam.sdk.io.splunk.SplunkEvent} or writing to HEC will be outputted to a GCS link. + */ +public final class GCSToSplunk { + + /** String/String Coder for FailsafeElement. */ + private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + /** The tag for the headers of the CSV if required. */ + private static final TupleTag CSV_HEADERS = new TupleTag() {}; + + /** The tag for the lines of the CSV. */ + private static final TupleTag CSV_LINES = new TupleTag() {}; + + /** The tag for the main output for the UDF. */ + @VisibleForTesting + static final TupleTag> UDF_OUT = + new TupleTag>() {}; + + /** The tag for the error output of the udf. */ + @VisibleForTesting + static final TupleTag> UDF_ERROR_OUT = + new TupleTag>() {}; + + /** The tag for successful {@link SplunkEvent} conversion. */ + @VisibleForTesting + static final TupleTag SPLUNK_EVENT_OUT = new TupleTag() {}; + + /** The tag for failed {@link SplunkEvent} conversion. */ + @VisibleForTesting + static final TupleTag> SPLUNK_EVENT_ERROR_OUT = + new TupleTag>() {}; + + /** The tag for all the elements that failed. */ + @VisibleForTesting static final TupleTag COMBINED_ERRORS = new TupleTag() {}; + + /** Logger for class. */ + private static final Logger LOG = LoggerFactory.getLogger(GCSToSplunk.class); + + /** + * The {@link GCSToSplunkOptions} class provides the custom execution options passed by the + * executor at the command-line. + */ + public interface GCSToSplunkOptions extends CsvConverters.CsvPipelineOptions, SplunkOptions { + + @Description("Pattern of where to write errors, ex: gs://mybucket/somepath/errors.txt") + String getInvalidOutputPath(); + + void setInvalidOutputPath(String value); + } + + /** + * Main entry-point for the pipeline. Reads in the command-line arguments, parses them, and + * executes the pipeline. + * + * @param args Arguments passed in from the command-line. + */ + public static void main(String[] args) { + GCSToSplunkOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(GCSToSplunkOptions.class); + + run(options); + } + + /** + * Runs the pipeline to completion with the specified options. + * + * @param options The execution options. + * @return The pipeline result. + */ + public static PipelineResult run(GCSToSplunkOptions options) { + Pipeline pipeline = Pipeline.create(options); + + CoderRegistry registry = pipeline.getCoderRegistry(); + registry.registerCoderForClass(SplunkEvent.class, SplunkEventCoder.of()); + registry.registerCoderForType( + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + + PCollectionTuple readCsvTuple = pipeline.apply("Read CSV", readFromCsv(options)); + + PCollectionTuple failsafeTransformedLines = + readCsvTuple.apply("Convert To JSON", convertToFailsafeAndMaybeApplyUdf(options)); + + PCollectionTuple splunkEventTuple = + failsafeTransformedLines + .get(UDF_OUT) + .apply("Convert to Splunk Event", convertToSplunkEvent()); + + PCollection wrappedSplunkWriteErrors = + splunkEventTuple.get(SPLUNK_EVENT_OUT).apply("Write to Splunk", writeToSplunk(options)); + + flattenErrorsAndConvertToString( + failsafeTransformedLines.get(UDF_ERROR_OUT), + splunkEventTuple.get(SPLUNK_EVENT_ERROR_OUT), + wrappedSplunkWriteErrors) + .apply("Output Errors To GCS", writeErrorsToGCS(options)); + + return pipeline.run(); + } + + static ReadCsv readFromCsv(GCSToSplunkOptions options) { + return CsvConverters.ReadCsv.newBuilder() + .setCsvFormat(options.getCsvFormat()) + .setDelimiter(options.getDelimiter()) + .setHasHeaders(options.getContainsHeaders()) + .setInputFileSpec(options.getInputFileSpec()) + .setHeaderTag(CSV_HEADERS) + .setLineTag(CSV_LINES) + .setFileEncoding(options.getCsvFileEncoding()) + .build(); + } + + static LineToFailsafeJson convertToFailsafeAndMaybeApplyUdf(GCSToSplunkOptions options) { + return CsvConverters.LineToFailsafeJson.newBuilder() + .setDelimiter(options.getDelimiter()) + .setUdfFileSystemPath(options.getJavascriptTextTransformGcsPath()) + .setUdfFunctionName(options.getJavascriptTextTransformFunctionName()) + .setJsonSchemaPath(options.getJsonSchemaPath()) + .setHeaderTag(CSV_HEADERS) + .setLineTag(CSV_LINES) + .setUdfOutputTag(UDF_OUT) + .setUdfDeadletterTag(UDF_ERROR_OUT) + .build(); + } + + static FailsafeStringToSplunkEvent convertToSplunkEvent() { + return SplunkConverters.failsafeStringToSplunkEvent(SPLUNK_EVENT_OUT, SPLUNK_EVENT_ERROR_OUT); + } + + static SplunkIO.Write writeToSplunk(GCSToSplunkOptions options) { + return SplunkIO.write(options.getUrl(), options.getToken()) + .withBatchCount(options.getBatchCount()) + .withParallelism(options.getParallelism()) + .withDisableCertificateValidation(options.getDisableCertificateValidation()); + } + + static PCollectionTuple flattenErrorsAndConvertToString( + PCollection> failsafeFailedTransformedLines, + PCollection> splunkEventFailedTransformedLines, + PCollection splunkWriteErrors) { + PCollection> wrappedSplunkWriteErrors = + splunkWriteErrors.apply( + "Wrap Splunk Write Errors", ParDo.of(new SplunkWriteErrorToFailsafeElementDoFn())); + + return PCollectionTuple.of( + COMBINED_ERRORS, + PCollectionList.of( + ImmutableList.of( + failsafeFailedTransformedLines, + wrappedSplunkWriteErrors, + splunkEventFailedTransformedLines)) + .apply("Flatten Errors", Flatten.pCollections()) + .apply("Convert Errors To String", ParDo.of(new FailsafeElementToStringDoFn()))); + } + + static LogErrors writeErrorsToGCS(GCSToSplunkOptions options) { + return LogErrors.newBuilder() + .setErrorWritePath(options.getInvalidOutputPath()) + .setErrorTag(COMBINED_ERRORS) + .build(); + } + + private static class SplunkWriteErrorToFailsafeElementDoFn + extends DoFn> { + @ProcessElement + public void processElement(ProcessContext context) { + SplunkWriteError error = context.element(); + FailsafeElement failsafeElement = + FailsafeElement.of(error.payload(), error.payload()); + + String statusMessage = ""; + if (error.statusMessage() != null) { + failsafeElement.setErrorMessage(error.statusMessage()); + statusMessage = failsafeElement.getErrorMessage(); + } + + if (error.statusCode() != null) { + failsafeElement.setErrorMessage( + statusMessage + ". Splunk write status code: " + error.statusCode()); + } + context.output(failsafeElement); + } + } + + private static class FailsafeElementToStringDoFn + extends DoFn, String> { + @ProcessElement + public void processElement(ProcessContext context) { + FailsafeElement error = context.element(); + String errorString = ""; + + if (error.getPayload() != null) { + errorString += "Payload: " + error.getPayload() + ". "; + } + + if (error.getErrorMessage() != null) { + errorString += "Error Message: " + error.getErrorMessage() + "."; + } + context.output(errorString); + } + } +} diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java new file mode 100644 index 0000000000..4fbd1431e0 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/templates/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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. + */ + +/** Google Cloud to Splunk template for Google Cloud Teleport. */ +package com.google.cloud.teleport.v2.templates; diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/SplunkConverters.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/SplunkConverters.java new file mode 100644 index 0000000000..a7cf25caf9 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/SplunkConverters.java @@ -0,0 +1,266 @@ +/* + * Copyright (C) 2022 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.v2.transforms; + +import com.google.api.client.util.DateTime; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.common.base.Throwables; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import com.google.gson.JsonSyntaxException; +import org.apache.beam.sdk.io.splunk.SplunkEvent; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Collection of utility {@link PTransform}s, {@link DoFn} and {@link PipelineOptions} used by + * pipelines that process and sink data using {@link org.apache.beam.sdk.io.splunk.SplunkIO}. + */ +public final class SplunkConverters { + + private static final Logger LOG = LoggerFactory.getLogger(SplunkConverters.class); + + private static final JsonParser jsonParser = new JsonParser(); + + // Key for grouping metadata fields. + private static final String METADATA_KEY = "_metadata"; + + // Users may use a UDF to add additional fields such as host and index + // that can be used for routing messages to different indexes in HEC. + // These metadata fields should be added in a nested JsonObject corresponding + // to key _metadata. + private static final String HEC_EVENT_KEY = "event"; + private static final String HEC_HOST_KEY = "host"; + private static final String HEC_INDEX_KEY = "index"; + private static final String HEC_TIME_KEY = "time"; + private static final String HEC_SOURCE_KEY = "source"; + private static final String HEC_SOURCE_TYPE_KEY = "sourcetype"; + private static final String TIMESTAMP_KEY = "timestamp"; + + /** + * Returns a {@link FailsafeStringToSplunkEvent} {@link PTransform} that consumes {@link + * FailsafeElement} messages, attempts to parse it as a JSON and extract metadata fields needed by + * Splunk's HEC endpoint (e.g. host, index etc) and creates {@link SplunkEvent} objects. Any + * conversion errors are wrapped into a {@link FailsafeElement} with appropriate error + * information. + * + * @param splunkEventOutputTag {@link TupleTag} to use for successfully converted messages. + * @param splunkErrorTag {@link TupleTag} to use for messages that failed conversion. + */ + public static FailsafeStringToSplunkEvent failsafeStringToSplunkEvent( + TupleTag splunkEventOutputTag, + TupleTag> splunkErrorTag) { + return new FailsafeStringToSplunkEvent(splunkEventOutputTag, splunkErrorTag); + } + + /** + * The {@link SplunkOptions} class provides the custom options passed by the executor at the + * command line. + */ + public interface SplunkOptions extends PipelineOptions { + + @Description("Splunk Http Event Collector (HEC) authentication token.") + String getToken(); + + void setToken(String token); + + @Description( + "Splunk Http Event Collector (HEC) url. " + + "This should be routable from the VPC in which the Dataflow pipeline runs. " + + "e.g. http://splunk-hec-host:8088") + String getUrl(); + + void setUrl(String url); + + @Description( + "Batch count for sending multiple events to " + + "Splunk's Http Event Collector in a single POST.") + Integer getBatchCount(); + + void setBatchCount(Integer batchCount); + + @Description("Disable SSL certificate validation.") + Boolean getDisableCertificateValidation(); + + void setDisableCertificateValidation(Boolean disableCertificateValidation); + + @Description("Maximum number of parallel requests.") + Integer getParallelism(); + + void setParallelism(Integer parallelism); + } + + /** + * The {@link FailsafeStringToSplunkEvent} class is a {@link PTransform} that returns a {@link + * PCollectionTuple} consisting of the following {@link PCollection}: + * + *
    + *
  • {@link FailsafeStringToSplunkEvent#splunkEventOutputTag()} - Contains {@link SplunkEvent} + * objects converted from input. + *
  • {@link FailsafeStringToSplunkEvent#splunkEventErrorTag()} - Contains {@link + * FailsafeElement} objects of conversion failures. + *
+ */ + public static final class FailsafeStringToSplunkEvent + extends PTransform>, PCollectionTuple> { + + private static final Counter CONVERSION_ERRORS = + Metrics.counter(FailsafeStringToSplunkEvent.class, "splunk-event-conversion-errors"); + + private static final Counter CONVERSION_SUCCESS = + Metrics.counter(FailsafeStringToSplunkEvent.class, "splunk-event-conversion-successes"); + + private TupleTag splunkEventOutputTag; + private TupleTag> splunkEventErrorTag; + + FailsafeStringToSplunkEvent( + TupleTag splunkEventOutputTag, + TupleTag> splunkEventErrorTag) { + this.splunkEventOutputTag = splunkEventOutputTag; + this.splunkEventErrorTag = splunkEventErrorTag; + } + + @Override + public PCollectionTuple expand(PCollection> input) { + + return input.apply( + "Convert To Splunk Event", + ParDo.of( + new DoFn, SplunkEvent>() { + + @ProcessElement + public void processElement(ProcessContext context) { + + String input = context.element().getPayload(); + + try { + + // Start building a SplunkEvent with the payload as the event. + SplunkEvent.Builder builder = SplunkEvent.newBuilder().withEvent(input); + + // We will attempt to parse the input to see + // if it is a valid JSON and if so, whether we can + // extract some additional properties that would be + // present in Stackdriver's LogEntry structure (timestamp) or + // a user provided _metadata field. + try { + + JsonObject json = jsonParser.parse(input).getAsJsonObject(); + + // Check if metadata is provided via a nested _metadata + // JSON object + JsonObject metadata = json.getAsJsonObject(METADATA_KEY); + + // We attempt to extract the timestamp from TIMESTAMP_KEY (if available) + // only if metadata does not exist. + String parsedTimestamp = ""; + if (metadata == null && json.get(TIMESTAMP_KEY) != null) { + parsedTimestamp = json.get(TIMESTAMP_KEY).getAsString(); + } + + // For the metadata fields, we only look at the _metadata + // object if present. + if (metadata != null) { + if (metadata.get(HEC_TIME_KEY) != null) { + parsedTimestamp = metadata.get(HEC_TIME_KEY).getAsString(); + } + + JsonElement source = metadata.get(HEC_SOURCE_KEY); + if (source != null) { + builder.withSource(source.getAsString()); + } + + JsonElement sourceType = metadata.get(HEC_SOURCE_TYPE_KEY); + if (sourceType != null) { + builder.withSourceType(sourceType.getAsString()); + } + + JsonElement host = metadata.get(HEC_HOST_KEY); + if (host != null) { + builder.withHost(host.getAsString()); + } + + JsonElement index = metadata.get(HEC_INDEX_KEY); + if (index != null) { + builder.withIndex(index.getAsString()); + } + + JsonElement event = metadata.get(HEC_EVENT_KEY); + if (event != null) { + builder.withEvent(event.getAsString()); + } + + // We remove the _metadata entry from the payload + // to avoid duplicates in Splunk. The relevant entries + // have been parsed and populated in the SplunkEvent metadata. + json.remove(METADATA_KEY); + // If the event was not overridden in metadata above, use + // the received JSON as event. + if (event == null) { + builder.withEvent(json.toString()); + } + } + + if (!parsedTimestamp.isEmpty()) { + try { + builder.withTime(DateTime.parseRfc3339(parsedTimestamp).getValue()); + } catch (NumberFormatException n) { + // We log this exception but don't want to fail the entire record. + LOG.warn( + "Unable to parse non-rfc3339 formatted timestamp: {}", + parsedTimestamp); + } + } + } catch (IllegalStateException | JsonSyntaxException e) { + // input is either not a properly formatted JSONObject + // or has other exceptions. In this case, we will + // simply capture the entire input as an 'event' and + // not worry about capturing any specific properties + // (for e.g Timestamp etc). + // We also do not want to LOG this as we might be running + // a pipeline to simply log text entries to Splunk and + // this is expected behavior. + } + + context.output(splunkEventOutputTag, builder.create()); + CONVERSION_SUCCESS.inc(); + + } catch (Exception e) { + CONVERSION_ERRORS.inc(); + context.output( + splunkEventErrorTag, + FailsafeElement.of(input, input) + .setErrorMessage(e.getMessage()) + .setStacktrace(Throwables.getStackTraceAsString(e))); + } + } + }) + .withOutputTags(splunkEventOutputTag, TupleTagList.of(splunkEventErrorTag))); + } + } +} diff --git a/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java new file mode 100644 index 0000000000..9c61a04bba --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/java/com/google/cloud/teleport/v2/transforms/package-info.java @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2022 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. + */ + +/** Google Cloud to Splunk transforms for Google Cloud Teleport. */ +package com.google.cloud.teleport.v2.transforms; diff --git a/v2/googlecloud-to-splunk/src/main/resources/gcs-to-splunk-command-spec.json b/v2/googlecloud-to-splunk/src/main/resources/gcs-to-splunk-command-spec.json new file mode 100644 index 0000000000..8b3e18b4d0 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/main/resources/gcs-to-splunk-command-spec.json @@ -0,0 +1,7 @@ +{ + "mainClass": "com.google.cloud.teleport.v2.templates.GCSToSplunk", + "classPath": "/template/gcs-to-splunk/*:/template/gcs-to-splunk/libs/*:/template/gcs-to-splunk/classes", + "defaultParameterValues": { + "labels": "{\"goog-dataflow-provided-template-type\":\"flex\",\"goog-dataflow-provided-template-name\":\"gcs-to-splunk\"}" + } +} \ No newline at end of file diff --git a/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkEventCoderTest.java b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkEventCoderTest.java new file mode 100644 index 0000000000..cb1f7846f3 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkEventCoderTest.java @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2022 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.v2.coders; + +import static com.google.common.truth.Truth.assertThat; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.beam.sdk.io.splunk.SplunkEvent; +import org.junit.Test; + +/** Unit tests for {@link SplunkEventCoder} class. */ +public final class SplunkEventCoderTest { + + @Test + public void testEncodeDecode() throws IOException { + String event = "test-event"; + String host = "test-host"; + String index = "test-index"; + String source = "test-source"; + String sourceType = "test-source-type"; + Long time = 123456789L; + + SplunkEvent actualEvent = + SplunkEvent.newBuilder() + .withEvent(event) + .withHost(host) + .withIndex(index) + .withSource(source) + .withSourceType(sourceType) + .withTime(time) + .create(); + + SplunkEventCoder coder = SplunkEventCoder.of(); + try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { + coder.encode(actualEvent, bos); + try (ByteArrayInputStream bin = new ByteArrayInputStream(bos.toByteArray())) { + SplunkEvent decodedEvent = coder.decode(bin); + assertThat(decodedEvent).isEqualTo(actualEvent); + } + } + } +} diff --git a/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoderTest.java b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoderTest.java new file mode 100644 index 0000000000..e1d01fb1ff --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/coders/SplunkWriteErrorCoderTest.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2022 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.v2.coders; + +import static com.google.common.truth.Truth.assertThat; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.beam.sdk.io.splunk.SplunkWriteError; +import org.junit.Test; + +/** Unit tests for {@link SplunkWriteErrorCoder} class. */ +public final class SplunkWriteErrorCoderTest { + + @Test + public void testEncodeDecode() throws IOException { + String payload = "test-payload"; + String message = "test-message"; + Integer statusCode = 123; + + SplunkWriteError actualError = + SplunkWriteError.newBuilder() + .withPayload(payload) + .withStatusCode(statusCode) + .withStatusMessage(message) + .create(); + + SplunkWriteErrorCoder coder = SplunkWriteErrorCoder.of(); + try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { + coder.encode(actualError, bos); + try (ByteArrayInputStream bin = new ByteArrayInputStream(bos.toByteArray())) { + SplunkWriteError decodedWriteError = coder.decode(bin); + assertThat(decodedWriteError).isEqualTo(actualError); + } + } + } +} diff --git a/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/templates/GCSToSplunkTest.java b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/templates/GCSToSplunkTest.java new file mode 100644 index 0000000000..d9749d3f31 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/templates/GCSToSplunkTest.java @@ -0,0 +1,276 @@ +/* + * Copyright (C) 2022 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.v2.templates; + +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.COMBINED_ERRORS; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.SPLUNK_EVENT_ERROR_OUT; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.SPLUNK_EVENT_OUT; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.UDF_ERROR_OUT; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.UDF_OUT; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.convertToFailsafeAndMaybeApplyUdf; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.convertToSplunkEvent; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.flattenErrorsAndConvertToString; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.readFromCsv; +import static com.google.cloud.teleport.v2.templates.GCSToSplunk.writeErrorsToGCS; +import static com.google.common.truth.Truth.assertThat; + +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; +import com.google.cloud.teleport.v2.coders.SplunkEventCoder; +import com.google.cloud.teleport.v2.coders.SplunkWriteErrorCoder; +import com.google.cloud.teleport.v2.templates.GCSToSplunk.GCSToSplunkOptions; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import com.google.common.io.Resources; +import java.io.File; +import java.io.IOException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.splunk.SplunkEvent; +import org.apache.beam.sdk.io.splunk.SplunkWriteError; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** Test cases for the {@link GCSToSplunk} class. */ +public final class GCSToSplunkTest { + @Rule public final TestPipeline pipeline = TestPipeline.create(); + + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + + private static final String TRANSFORM_FILE_PATH = Resources.getResource("splunkUdf.js").getPath(); + private static final String NO_HEADER_CSV_FILE_PATH = + Resources.getResource("no_header.csv").getPath(); + private static final String HEADER_CSV_FILE_PATH = + Resources.getResource("with_headers.csv").getPath(); + private static final String JSON_SCHEMA_FILE_PATH = + Resources.getResource("testSchema.json").getPath(); + + @Test + public void testGCSToSplunkReadUdf() { + // Arrange + String stringifiedJsonRecord = "{\"id\":\"007\",\"state\":\"CA\",\"price\":26.23}"; + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder().withEvent(stringifiedJsonRecord).create(); + + CoderRegistry coderRegistry = pipeline.getCoderRegistry(); + coderRegistry.registerCoderForClass(SplunkEvent.class, SplunkEventCoder.of()); + coderRegistry.registerCoderForType( + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + + GCSToSplunkOptions options = PipelineOptionsFactory.create().as(GCSToSplunkOptions.class); + + options.setJavascriptTextTransformGcsPath(TRANSFORM_FILE_PATH); + options.setJavascriptTextTransformFunctionName("transform"); + options.setContainsHeaders(false); + options.setInputFileSpec(NO_HEADER_CSV_FILE_PATH); + + // Act + PCollectionTuple readCsvOut = pipeline.apply("Read CSV", readFromCsv(options)); + PCollectionTuple transformedLines = + readCsvOut.apply("Convert to JSON", convertToFailsafeAndMaybeApplyUdf(options)); + PCollectionTuple splunkEventTuple = + transformedLines.get(UDF_OUT).apply("Convert to Splunk Event", convertToSplunkEvent()); + + // Assert + PAssert.that(transformedLines.get(UDF_OUT)) + .satisfies( + collection -> { + FailsafeElement element = collection.iterator().next(); + assertThat(element.getPayload()).isEqualTo(stringifiedJsonRecord); + return null; + }); + PAssert.that(transformedLines.get(UDF_ERROR_OUT)).empty(); + PAssert.that(splunkEventTuple.get(SPLUNK_EVENT_OUT)).containsInAnyOrder(expectedSplunkEvent); + PAssert.that(splunkEventTuple.get(SPLUNK_EVENT_ERROR_OUT)).empty(); + + // Execute pipeline + pipeline.run(); + } + + @Test + public void testGCSToSplunkReadHeaders() { + // Arrange + String stringifiedJsonRecord = "{\"id\":\"008\",\"state\":\"CA\",\"price\":\"26.23\"}"; + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder().withEvent(stringifiedJsonRecord).create(); + + CoderRegistry coderRegistry = pipeline.getCoderRegistry(); + coderRegistry.registerCoderForClass(SplunkEvent.class, SplunkEventCoder.of()); + coderRegistry.registerCoderForType( + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + + GCSToSplunkOptions options = PipelineOptionsFactory.create().as(GCSToSplunkOptions.class); + + options.setContainsHeaders(true); + options.setInputFileSpec(HEADER_CSV_FILE_PATH); + + // Act + PCollectionTuple readCsvOut = pipeline.apply("Read CSV", readFromCsv(options)); + PCollectionTuple transformedLines = + readCsvOut.apply("Convert to JSON", convertToFailsafeAndMaybeApplyUdf(options)); + PCollectionTuple splunkEventTuple = + transformedLines.get(UDF_OUT).apply("Convert to Splunk Event", convertToSplunkEvent()); + + // Assert + PAssert.that(transformedLines.get(UDF_OUT)) + .satisfies( + collection -> { + FailsafeElement element = collection.iterator().next(); + assertThat(element.getPayload()).isEqualTo(stringifiedJsonRecord); + return null; + }); + PAssert.that(transformedLines.get(UDF_ERROR_OUT)).empty(); + PAssert.that(splunkEventTuple.get(SPLUNK_EVENT_OUT)).containsInAnyOrder(expectedSplunkEvent); + PAssert.that(splunkEventTuple.get(SPLUNK_EVENT_ERROR_OUT)).empty(); + + // Execute pipeline + pipeline.run(); + } + + @Test + public void testGCSToSplunkReadJsonSchema() { + // Arrange + String stringifiedJsonRecord = "{\"id\":\"007\",\"state\":\"CA\",\"price\":26.23}"; + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder().withEvent(stringifiedJsonRecord).create(); + + CoderRegistry coderRegistry = pipeline.getCoderRegistry(); + coderRegistry.registerCoderForClass(SplunkEvent.class, SplunkEventCoder.of()); + coderRegistry.registerCoderForType( + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + + GCSToSplunkOptions options = PipelineOptionsFactory.create().as(GCSToSplunkOptions.class); + + options.setJsonSchemaPath(JSON_SCHEMA_FILE_PATH); + options.setContainsHeaders(false); + options.setInputFileSpec(NO_HEADER_CSV_FILE_PATH); + + // Act + PCollectionTuple readCsvOut = pipeline.apply("Read CSV", readFromCsv(options)); + PCollectionTuple transformedLines = + readCsvOut.apply("Convert to JSON", convertToFailsafeAndMaybeApplyUdf(options)); + PCollectionTuple splunkEventTuple = + transformedLines.get(UDF_OUT).apply("Convert to Splunk Event", convertToSplunkEvent()); + + // Assert + PAssert.that(transformedLines.get(UDF_OUT)) + .satisfies( + collection -> { + FailsafeElement element = collection.iterator().next(); + assertThat(element.getPayload()).isEqualTo(stringifiedJsonRecord); + return null; + }); + PAssert.that(transformedLines.get(UDF_ERROR_OUT)).empty(); + PAssert.that(splunkEventTuple.get(SPLUNK_EVENT_OUT)).containsInAnyOrder(expectedSplunkEvent); + PAssert.that(splunkEventTuple.get(SPLUNK_EVENT_ERROR_OUT)).empty(); + + // Execute pipeline + pipeline.run(); + } + + @Test + public void testGCSToSplunkConvertWriteErrors() { + // Arrange + String stringifiedSplunkError = + "Payload: test-payload. Error Message: test-message. Splunk write status code: 123."; + String firstStringifiedFailsafeError = "Payload: world. Error Message: failed!."; + String secondStringifiedFailsafeError = "Payload: one. Error Message: error!."; + + CoderRegistry coderRegistry = pipeline.getCoderRegistry(); + coderRegistry.registerCoderForType( + FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor(), FAILSAFE_ELEMENT_CODER); + + SplunkWriteError splunkWriteError = + SplunkWriteError.newBuilder() + .withPayload("test-payload") + .withStatusCode(123) + .withStatusMessage("test-message") + .create(); + + PCollection splunkErrorCollection = + pipeline.apply( + "Add Splunk Errors", Create.of(splunkWriteError).withCoder(SplunkWriteErrorCoder.of())); + + FailsafeElement firstFailsafeElement = + FailsafeElement.of("hello", "world").setErrorMessage("failed!"); + + PCollection> firstFailsafeElementCollection = + pipeline.apply( + "Add FailsafeElements to First", + Create.of(firstFailsafeElement).withCoder(FAILSAFE_ELEMENT_CODER)); + + FailsafeElement secondFailsafeElement = + FailsafeElement.of("another", "one").setErrorMessage("error!"); + + PCollection> secondFailsafeElementCollection = + pipeline.apply( + "Add FailsafeElements to Second", + Create.of(secondFailsafeElement).withCoder(FAILSAFE_ELEMENT_CODER)); + + // Act + PCollectionTuple stringifiedErrors = + flattenErrorsAndConvertToString( + firstFailsafeElementCollection, secondFailsafeElementCollection, splunkErrorCollection); + + // Assert + PAssert.that(stringifiedErrors.get(COMBINED_ERRORS)) + .containsInAnyOrder( + stringifiedSplunkError, firstStringifiedFailsafeError, secondStringifiedFailsafeError); + + // Execute pipeline + pipeline.run(); + } + + @Test + public void testGCSToSplunkWriteErrorsToFolder() throws IOException { + // Arrange + String stringifiedSplunkError = + "Payload: test-payload. Error Message: test-message. Splunk write status code: 123."; + + PCollection stringifiedErrorCollection = + pipeline.apply( + "Add Stringified Errors", + Create.of(stringifiedSplunkError).withCoder(StringUtf8Coder.of())); + + PCollectionTuple stringifiedErrorTuple = + PCollectionTuple.of(COMBINED_ERRORS, stringifiedErrorCollection); + + GCSToSplunkOptions options = PipelineOptionsFactory.create().as(GCSToSplunkOptions.class); + + options.setInvalidOutputPath(tmpFolder.getRoot().getAbsolutePath() + "errors.txt"); + + // Act + stringifiedErrorTuple.apply("Output Errors To GCS", writeErrorsToGCS(options)); + + // Execute pipeline + pipeline.run(); + + // Assert + File file = new File(tmpFolder.getRoot().getAbsolutePath() + "errors.txt-00000-of-00001"); + String fileContents = Files.toString(file, Charsets.UTF_8); + assertThat(fileContents).contains(stringifiedSplunkError); + } +} diff --git a/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/transforms/SplunkConvertersTest.java b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/transforms/SplunkConvertersTest.java new file mode 100644 index 0000000000..3cccc0296d --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/java/com/google/cloud/teleport/v2/transforms/SplunkConvertersTest.java @@ -0,0 +1,247 @@ +/* + * Copyright (C) 2022 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.v2.transforms; + +import com.google.api.client.util.DateTime; +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; +import com.google.cloud.teleport.v2.coders.SplunkEventCoder; +import com.google.cloud.teleport.v2.values.FailsafeElement; +import com.google.gson.Gson; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.splunk.SplunkEvent; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** Unit tests for {@link SplunkConverters} class. */ +public final class SplunkConvertersTest { + + private static final TupleTag SPLUNK_EVENT_OUT = new TupleTag() {}; + private static final TupleTag> SPLUNK_EVENT_ERROR_OUT = + new TupleTag>() {}; + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static final Gson GSON = new Gson(); + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventSimpleStrings() { + FailsafeElement hello = FailsafeElement.of("hello", "hello"); + FailsafeElement world = FailsafeElement.of("world", "world"); + + pipeline.getCoderRegistry().registerCoderForClass(SplunkEvent.class, SplunkEventCoder.of()); + + PCollectionTuple tuple = + pipeline + .apply( + Create.of(hello, world) + .withCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))) + .apply( + SplunkConverters.failsafeStringToSplunkEvent( + SPLUNK_EVENT_OUT, SPLUNK_EVENT_ERROR_OUT)); + + PAssert.that(tuple.get(SPLUNK_EVENT_ERROR_OUT)).empty(); + PAssert.that(tuple.get(SPLUNK_EVENT_OUT)) + .containsInAnyOrder( + SplunkEvent.newBuilder().withEvent("hello").create(), + SplunkEvent.newBuilder().withEvent("world").create()); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventInvalidJSON() { + FailsafeElement input = + FailsafeElement.of( + "" + "\t\"name\": \"Jim\",\n" + "}", "{\n" + "\t\"name\": \"Jim\",\n" + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder().withEvent("{\n" + "\t\"name\": \"Jim\",\n" + "}").create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventValidJSON() { + FailsafeElement input = + FailsafeElement.of( + "" + "\t\"name\": \"Jim\",\n" + "}", "{\n" + "\t\"name\": \"Jim\"\n" + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder().withEvent("{\n" + "\t\"name\": \"Jim\"\n" + "}").create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventValidTimestamp() { + FailsafeElement input = + FailsafeElement.of( + "", + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"logName\": \"test-log-name\",\n" + + "\t\"timestamp\": \"2019-10-15T11:32:26.553Z\"\n" + + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder() + .withEvent( + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"logName\": \"test-log-name\",\n" + + "\t\"timestamp\": \"2019-10-15T11:32:26.553Z\"\n" + + "}") + .withTime(DateTime.parseRfc3339("2019-10-15T11:32:26.553Z").getValue()) + .create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventInValidTimestamp() { + FailsafeElement input = + FailsafeElement.of( + "", + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"logName\": \"test-log-name\",\n" + + "\t\"timestamp\": \"2019-1011:32:26.553Z\"\n" + + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder() + .withEvent( + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"logName\": \"test-log-name\",\n" + + "\t\"timestamp\": \"2019-1011:32:26.553Z\"\n" + + "}") + .create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventValidSource() { + FailsafeElement input = + FailsafeElement.of( + "", + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"_metadata\": {\"source\": \"test-log-name\"}\n" + + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder() + .withEvent("{\"name\":\"Jim\"}") + .withSource("test-log-name") + .create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventValidHost() { + FailsafeElement input = + FailsafeElement.of( + "", + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"_metadata\": {\"host\": \"test-host\"}\n" + + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder().withEvent("{\"name\":\"Jim\"}").withHost("test-host").create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventValidIndex() { + FailsafeElement input = + FailsafeElement.of( + "", + "{\n" + + "\t\"name\": \"Jim\",\n" + + "\t\"_metadata\": {\"host\": \"test-host\"," + + "\"index\":\"test-index\"}\n" + + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder() + .withEvent("{\"name\":\"Jim\"}") + .withHost("test-host") + .withIndex("test-index") + .create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + @Test + @Category(NeedsRunner.class) + public void testFailsafeStringToSplunkEventValidTimeOverride() { + FailsafeElement input = + FailsafeElement.of( + "", + "{\n" + + "\t\"timestamp\": \"2019-10-15T11:32:26.553Z\",\n" + + "\t\"_metadata\": {\"time\": \"2019-11-22T11:32:26.553Z\", " + + "\"source\": \"test-source-name\"}\n" + + "}"); + + SplunkEvent expectedSplunkEvent = + SplunkEvent.newBuilder() + .withEvent("{" + "\"timestamp\":\"2019-10-15T11:32:26.553Z\"" + "}") + .withSource("test-source-name") + .withTime(DateTime.parseRfc3339("2019-11-22T11:32:26.553Z").getValue()) + .create(); + + matchesSplunkEvent(input, expectedSplunkEvent); + } + + private void matchesSplunkEvent( + FailsafeElement input, SplunkEvent expectedSplunkEvent) { + pipeline.getCoderRegistry().registerCoderForClass(SplunkEvent.class, SplunkEventCoder.of()); + + PCollectionTuple tuple = + pipeline + .apply( + Create.of(input) + .withCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))) + .apply( + SplunkConverters.failsafeStringToSplunkEvent( + SPLUNK_EVENT_OUT, SPLUNK_EVENT_ERROR_OUT)); + + PAssert.that(tuple.get(SPLUNK_EVENT_ERROR_OUT)).empty(); + PAssert.that(tuple.get(SPLUNK_EVENT_OUT)).containsInAnyOrder(expectedSplunkEvent); + + pipeline.run(); + } +} diff --git a/v2/googlecloud-to-splunk/src/test/resources/no_header.csv b/v2/googlecloud-to-splunk/src/test/resources/no_header.csv new file mode 100644 index 0000000000..4560bd82a3 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/resources/no_header.csv @@ -0,0 +1 @@ +007,CA,26.23 \ No newline at end of file diff --git a/v2/googlecloud-to-splunk/src/test/resources/splunkUdf.js b/v2/googlecloud-to-splunk/src/test/resources/splunkUdf.js new file mode 100644 index 0000000000..2c84597d59 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/resources/splunkUdf.js @@ -0,0 +1,30 @@ +/* + * Copyright (C) 2022 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. + */ + +/** + * This function splits a csv and turns it into a pre-defined JSON. + * @param {string} line is a line from TextIO. + * @return {JSON} a JSON created after parsing the line. + */ +function transform(line) { + var split = line.split(","); + var obj = new Object(); + obj.id = split[0]; + obj.state = split[1]; + obj.price = parseFloat(split[2]); + var jsonString = JSON.stringify(obj); + return jsonString; +} \ No newline at end of file diff --git a/v2/googlecloud-to-splunk/src/test/resources/testSchema.json b/v2/googlecloud-to-splunk/src/test/resources/testSchema.json new file mode 100644 index 0000000000..492be692d1 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/resources/testSchema.json @@ -0,0 +1,14 @@ +[ + { + "name": "id", + "type": "text" + }, + { + "name": "state", + "type": "text" + }, + { + "name": "price", + "type": "double" + } +] \ No newline at end of file diff --git a/v2/googlecloud-to-splunk/src/test/resources/with_headers.csv b/v2/googlecloud-to-splunk/src/test/resources/with_headers.csv new file mode 100644 index 0000000000..d1ae7a4240 --- /dev/null +++ b/v2/googlecloud-to-splunk/src/test/resources/with_headers.csv @@ -0,0 +1,2 @@ +id,state,price +008,CA,26.23 \ No newline at end of file diff --git a/v2/pom.xml b/v2/pom.xml index 0fbd62f6c0..6cb8a799cc 100644 --- a/v2/pom.xml +++ b/v2/pom.xml @@ -681,6 +681,7 @@ file-format-conversion googlecloud-to-elasticsearch googlecloud-to-googlecloud + googlecloud-to-splunk kafka-to-bigquery kafka-to-gcs kudu-to-bigquery From 5ebdace3859c13969b877e01f11ed3ddee5c3c64 Mon Sep 17 00:00:00 2001 From: olegsa Date: Wed, 25 May 2022 17:26:35 -0700 Subject: [PATCH 145/145] Fix flaky tests due to non-deterministic lass loading order The CL makes the static init of AvroSinkWithJodaDatesConversion to always run after static init of AvroUtils. This is to ensure that AvroSinkWithJodaDatesConversion's type conversion always overrides AvroUtils' type conversion. PiperOrigin-RevId: 451052742 --- .../teleport/v2/io/AvroSinkWithJodaDatesConversion.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/AvroSinkWithJodaDatesConversion.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/AvroSinkWithJodaDatesConversion.java index f872626b70..b1cf1342c2 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/AvroSinkWithJodaDatesConversion.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/io/AvroSinkWithJodaDatesConversion.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.AvroIO.Sink; import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.schemas.utils.AvroUtils; import org.joda.time.ReadableInstant; /** @@ -49,6 +50,11 @@ public class AvroSinkWithJodaDatesConversion implements FileIO.Sink { static { + // Call any AvroUtils method to force AvroUtils initialization to ensure that AvroUtils static + // init runs before this static init deterministically. + AvroUtils.schemaCoder(Object.class); + + // override type conversion that was done by AvroUtils SpecificData.get().addLogicalTypeConversion(JodaDateTimeOrLongConversion.INSTANCE); GenericData.get().addLogicalTypeConversion(JodaDateTimeOrLongConversion.INSTANCE); ReflectData.get().addLogicalTypeConversion(JodaDateTimeOrLongConversion.INSTANCE);