From add3dd70f79d1cbd804b02a5ba4434413b9a0a0d Mon Sep 17 00:00:00 2001 From: Jeff Kinard Date: Wed, 27 Nov 2024 14:15:00 -0500 Subject: [PATCH 01/14] add KMS key support to BigQueryConverters (#2024) * add KMS key support to BigQueryConverters Signed-off-by: Jeffrey Kinard * address comments Signed-off-by: Jeffrey Kinard --------- Signed-off-by: Jeffrey Kinard --- .../v2/transforms/BigQueryConverters.java | 25 ++++++++++++++++--- .../templates/BigQueryToElasticsearchIT.java | 24 +++++++++++++++--- 2 files changed, 42 insertions(+), 7 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 5502aead88..116071d69f 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 @@ -78,6 +78,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.commons.text.StringSubstitutor; +import org.apache.parquet.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -241,6 +242,18 @@ public interface BigQueryReadOptions extends PipelineOptions { String getQueryTempDataset(); void setQueryTempDataset(String queryTempDataset); + + @TemplateParameter.KmsEncryptionKey( + order = 7, + optional = true, + description = "Google Cloud KMS key", + helpText = + "If reading from BigQuery using query source, use this Cloud KMS key to encrypt any temporary tables created.", + example = + "projects/your-project/locations/global/keyRings/your-keyring/cryptoKeys/your-key") + String getKMSEncryptionKey(); + + void setKMSEncryptionKey(String keyName); } /** @@ -340,11 +353,17 @@ public static ReadBigQuery.Builder newBuilder() { @Override public PCollection expand(PBegin pipeline) { + BigQueryIO.TypedRead readFunction = readFunction(); + + if (!Strings.isNullOrEmpty(options().getKMSEncryptionKey())) { + readFunction = readFunction.withKmsKey(options().getKMSEncryptionKey()); + } + if (options().getQuery() == null) { LOG.info("No query provided, reading directly from: " + options().getInputTableSpec()); return pipeline.apply( "ReadFromBigQuery", - readFunction() + readFunction .from(options().getInputTableSpec()) .withTemplateCompatibility() .withMethod(Method.DIRECT_READ)); @@ -357,7 +376,7 @@ public PCollection expand(PBegin pipeline) { LOG.info("Using Standard SQL"); return pipeline.apply( "ReadFromBigQueryWithQuery", - readFunction() + readFunction .fromQuery(options().getQuery()) .withTemplateCompatibility() .withQueryLocation(options().getQueryLocation()) @@ -368,7 +387,7 @@ public PCollection expand(PBegin pipeline) { LOG.info("Using Legacy SQL"); return pipeline.apply( "ReadFromBigQueryWithQuery", - readFunction() + readFunction .fromQuery(options().getQuery()) .withTemplateCompatibility() .withQueryLocation(options().getQueryLocation()) diff --git a/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/templates/BigQueryToElasticsearchIT.java b/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/templates/BigQueryToElasticsearchIT.java index 374aa2cd2d..e3b7abb4af 100644 --- a/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/templates/BigQueryToElasticsearchIT.java +++ b/v2/googlecloud-to-elasticsearch/src/test/java/com/google/cloud/teleport/v2/elasticsearch/templates/BigQueryToElasticsearchIT.java @@ -29,6 +29,7 @@ import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.StandardSQLTypeName; import com.google.cloud.bigquery.TableId; +import com.google.cloud.kms.v1.CryptoKey; import com.google.cloud.teleport.metadata.TemplateIntegrationTest; import java.io.IOException; import java.util.List; @@ -42,6 +43,7 @@ import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.bigquery.BigQueryResourceManager; import org.apache.beam.it.gcp.bigquery.utils.BigQueryTestUtil; +import org.apache.beam.it.gcp.kms.KMSResourceManager; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -57,6 +59,14 @@ public final class BigQueryToElasticsearchIT extends TemplateTestBase { private BigQueryResourceManager bigQueryClient; private ElasticsearchResourceManager elasticsearchResourceManager; + private KMSResourceManager kmsResourceManager; + + // Used by BigQueryIO to encrypt data in temporary tables + CryptoKey cryptoKey; + + private static final String KMS_REGION = "global"; + private static final String KEYRING_ID = "BigQueryToElasticsearch"; + private static final String CRYPTO_KEY_NAME = "key1"; // Define a set of parameters used to allow configuration of the test size being run. private static final String BIGQUERY_ID_COL = "test_id"; @@ -72,11 +82,16 @@ public final class BigQueryToElasticsearchIT extends TemplateTestBase { public void setup() { bigQueryClient = BigQueryResourceManager.builder(testName, PROJECT, credentials).build(); elasticsearchResourceManager = ElasticsearchResourceManager.builder(testId).build(); + kmsResourceManager = + KMSResourceManager.builder(PROJECT, credentialsProvider).setRegion(KMS_REGION).build(); + + cryptoKey = kmsResourceManager.getOrCreateCryptoKey(KEYRING_ID, CRYPTO_KEY_NAME); } @After public void tearDown() { - ResourceManagerUtils.cleanResources(bigQueryClient, elasticsearchResourceManager); + ResourceManagerUtils.cleanResources( + bigQueryClient, elasticsearchResourceManager, kmsResourceManager); } @Test @@ -111,7 +126,7 @@ public void testBigQueryToElasticsearch() throws IOException { // Assert assertThatResult(result).isLaunchFinished(); - assertThat(elasticsearchResourceManager.count(indexName)).isEqualTo(20); + assertThat(elasticsearchResourceManager.count(indexName)).isEqualTo(BIGQUERY_NUM_ROWS); assertThatRecords(elasticsearchResourceManager.fetchAll(indexName)) .hasRecordsUnordered(bigQueryRowsToRecords(bigQueryRows)); } @@ -143,7 +158,8 @@ public void testBigQueryToElasticsearchQuery() throws IOException { .addParameter("connectionUrl", elasticsearchResourceManager.getUri()) .addParameter("disableCertificateValidation", "true") .addParameter("index", indexName) - .addParameter("apiKey", "elastic")); + .addParameter("apiKey", "elastic") + .addParameter("KMSEncryptionKey", cryptoKey.getName())); assertThatPipeline(info).isRunning(); Result result = pipelineOperator().waitUntilDone(createConfig(info)); @@ -151,7 +167,7 @@ public void testBigQueryToElasticsearchQuery() throws IOException { // Assert assertThatResult(result).isLaunchFinished(); - assertThat(elasticsearchResourceManager.count(indexName)).isEqualTo(20); + assertThat(elasticsearchResourceManager.count(indexName)).isEqualTo(BIGQUERY_NUM_ROWS); assertThatRecords(elasticsearchResourceManager.fetchAll(indexName)) .hasRecordsUnordered(bigQueryRowsToRecords(bigQueryRows)); } From c5a74fce223ffec1c642e90ecedc1dee62356991 Mon Sep 17 00:00:00 2001 From: Vardhan Vinay Thigle <39047439+VardhanThigle@users.noreply.github.com> Date: Thu, 28 Nov 2024 09:40:24 +0000 Subject: [PATCH 02/14] Allowing Jdbc FetchSize Configuration for handling large rows (#2028) --- .../v2/options/OptionsToConfigBuilder.java | 7 ++- .../v2/options/SourceDbToSpannerOptions.java | 34 ++++++++----- .../auth/dbauth/LocalCredentialsProvider.java | 5 ++ .../io/jdbc/iowrapper/JdbcIoWrapper.java | 8 ++- .../transforms/ReadWithUniformPartitions.java | 47 +++++++++++++++--- .../v2/templates/PipelineController.java | 3 +- .../options/OptionsToConfigBuilderTest.java | 25 ++++++++-- .../io/jdbc/iowrapper/JdbcIoWrapperTest.java | 10 ++++ .../ReadWithUniformPartitionsTest.java | 49 +++++++++++++++++++ 9 files changed, 162 insertions(+), 26 deletions(-) diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java index 7c8ddb5b7c..57f12abc2d 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java @@ -76,7 +76,8 @@ public static JdbcIOWrapperConfig getJdbcIOWrapperConfigWithDefaults( jdbcDriverJars, maxConnections, numPartitions, - waitOn); + waitOn, + options.getFetchSize()); } public static JdbcIOWrapperConfig getJdbcIOWrapperConfig( @@ -95,7 +96,8 @@ public static JdbcIOWrapperConfig getJdbcIOWrapperConfig( String jdbcDriverJars, long maxConnections, Integer numPartitions, - Wait.OnSignal waitOn) { + Wait.OnSignal waitOn, + Integer fetchSize) { JdbcIOWrapperConfig.Builder builder = builderWithDefaultsFor(sqlDialect); SourceSchemaReference sourceSchemaReference = sourceSchemaReferenceFrom(sqlDialect, dbName, namespace); @@ -149,6 +151,7 @@ public static JdbcIOWrapperConfig getJdbcIOWrapperConfig( builder.setMaxPartitions(numPartitions); builder = builder.setTables(ImmutableList.copyOf(tables)); + builder = builder.setMaxFetchSize(fetchSize); return builder.build(); } diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java index d9321df03e..428010f91e 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java @@ -117,8 +117,20 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setNumPartitions(Integer value); - @TemplateParameter.Text( + @TemplateParameter.Integer( order = 9, + optional = true, + description = "The number of rows to fetch per page read for JDBC source.", + helpText = + "The number of rows to fetch per page read for JDBC source. If not set, the default of JdbcIO of 50_000 rows gets used." + + " This ultimately translated to Statement.setFetchSize call at Jdbc layer. It should ONLY be used if the default value throws memory errors." + + "In case you are using MySql source, please also note that FetchSize is ignored by the connector unless, you also set `useCursorFetch=true` as a connection property either in the source URL or the shard config file") + Integer getFetchSize(); + + void setFetchSize(Integer value); + + @TemplateParameter.Text( + order = 10, groupName = "Target", description = "Cloud Spanner Instance Id.", helpText = "The destination Cloud Spanner instance.") @@ -127,7 +139,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setInstanceId(String value); @TemplateParameter.Text( - order = 10, + order = 11, groupName = "Target", regexes = {"^[a-z]([a-z0-9_-]{0,28})[a-z0-9]$"}, description = "Cloud Spanner Database Id.", @@ -137,7 +149,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setDatabaseId(String value); @TemplateParameter.ProjectId( - order = 11, + order = 12, groupName = "Target", description = "Cloud Spanner Project Id.", helpText = "This is the name of the Cloud Spanner project.") @@ -146,7 +158,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setProjectId(String projectId); @TemplateParameter.Text( - order = 12, + order = 13, optional = true, description = "Cloud Spanner Endpoint to call", helpText = "The Cloud Spanner endpoint to call in the template.", @@ -157,7 +169,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setSpannerHost(String value); @TemplateParameter.Integer( - order = 13, + order = 14, optional = true, description = "Maximum number of connections to Source database per worker", helpText = @@ -169,7 +181,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setMaxConnections(Integer value); @TemplateParameter.GcsReadFile( - order = 14, + order = 15, optional = true, description = "Session File Path in Cloud Storage, to provide mapping information in the form of a session file", @@ -182,7 +194,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setSessionFilePath(String value); @TemplateParameter.GcsReadFile( - order = 15, + order = 16, description = "Output directory for failed/skipped/filtered events", helpText = "This directory is used to dump the failed/skipped/filtered records in a migration.") @@ -191,7 +203,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setOutputDirectory(String value); @TemplateParameter.GcsReadFile( - order = 16, + order = 17, optional = true, description = "Custom jar location in Cloud Storage", helpText = @@ -202,7 +214,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setTransformationJarPath(String value); @TemplateParameter.Text( - order = 17, + order = 18, optional = true, description = "Custom class name", helpText = @@ -214,7 +226,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setTransformationClassName(String value); @TemplateParameter.Text( - order = 18, + order = 19, optional = true, description = "Custom parameters for transformation", helpText = @@ -225,7 +237,7 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { void setTransformationCustomParameters(String value); @TemplateParameter.Text( - order = 19, + order = 20, optional = true, description = "Namespace", helpText = diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/auth/dbauth/LocalCredentialsProvider.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/auth/dbauth/LocalCredentialsProvider.java index e1aeff804a..aa33138144 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/auth/dbauth/LocalCredentialsProvider.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/auth/dbauth/LocalCredentialsProvider.java @@ -43,6 +43,11 @@ public static Builder builder() { return new AutoValue_LocalCredentialsProvider.Builder(); } + @Override + public String toString() { + return "LocalCredentialsProvider{}"; + } + @AutoValue.Builder public abstract static class Builder { diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapper.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapper.java index 2fe9c626e0..d001f8864f 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapper.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapper.java @@ -416,6 +416,9 @@ private static PTransform> getJdbcIO( if (tableConfig.maxPartitions() != null) { jdbcIO = jdbcIO.withNumPartitions(tableConfig.maxPartitions()); } + if (config.maxFetchSize() != null) { + jdbcIO = jdbcIO.withFetchSize(config.maxFetchSize()); + } return jdbcIO; } @@ -443,6 +446,7 @@ private static PTransform> getReadWithUniformPart .setDataSourceProviderFn(JdbcIO.PoolableDataSourceProvider.of(dataSourceConfiguration)) .setDbAdapter(config.dialectAdapter()) .setApproxTotalRowCount(tableConfig.approxRowCount()) + .setFetchSize(config.maxFetchSize()) .setRowMapper( new JdbcSourceRowMapper( config.valueMappingsProvider(), @@ -463,7 +467,9 @@ private static PTransform> getReadWithUniformPart readWithUniformPartitionsBuilder = readWithUniformPartitionsBuilder.setMaxPartitionsHint((long) tableConfig.maxPartitions()); } - return readWithUniformPartitionsBuilder.build(); + ReadWithUniformPartitions readWithUniformPartitions = readWithUniformPartitionsBuilder.build(); + LOG.info("Configured ReadWithUniformPartitions {} for {}", readWithUniformPartitions, config); + return readWithUniformPartitions; } /** diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitions.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitions.java index 38b92759a3..8941e4d30d 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitions.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitions.java @@ -34,6 +34,8 @@ import javax.sql.DataSource; import org.apache.beam.sdk.io.jdbc.JdbcIO; import org.apache.beam.sdk.io.jdbc.JdbcIO.PreparedStatementSetter; +import org.apache.beam.sdk.io.jdbc.JdbcIO.ReadAll; +import org.apache.beam.sdk.io.jdbc.JdbcIO.RowMapper; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; @@ -104,6 +106,14 @@ public abstract class ReadWithUniformPartitions extends PTransform rowMapper(); + /** + * Max fetch size for jdbc read. * If Null, {@link JdbcIO JdbcIO's} default fetch size of 50_000 + * gets used. {@link JdbcIO.Read#withFetchSize(int)} recommends setting this manually only if the + * default value gives out of memory errors. + */ + @Nullable + abstract Integer fetchSize(); + /** * Hint for Maximum number of partitions of the source key space. If not set, it is auto inferred * as 1/10 * sqrt({@link ReadWithUniformPartitions#autoAdjustMaxPartitions()}). Note that if @@ -221,12 +231,34 @@ public PCollection expand(PBegin input) { Reshuffle.viaRandomKey().withNumBuckets(dbParallelizationForReads())) .apply( getTransformName("RangeRead", null), - JdbcIO.readAll() - .withOutputParallelization(false) - .withQuery(dbAdapter().getReadQuery(tableName(), colNames)) - .withParameterSetter(rangePrepareator) - .withDataSourceProviderFn(dataSourceProviderFn()) - .withRowMapper(rowMapper())); + buildJdbcIO( + JdbcIO.readAll(), + dbAdapter().getReadQuery(tableName(), colNames), + rangePrepareator, + dataSourceProviderFn(), + rowMapper(), + fetchSize())); + } + + @VisibleForTesting + protected static JdbcIO.ReadAll buildJdbcIO( + JdbcIO.ReadAll readAll, + String readQuery, + PreparedStatementSetter rangePrepareator, + SerializableFunction dataSourceProviderFn, + RowMapper rowMapper, + Integer fetchSize) { + ReadAll ret = + readAll + .withOutputParallelization(false) + .withQuery(readQuery) + .withParameterSetter(rangePrepareator) + .withDataSourceProviderFn(dataSourceProviderFn) + .withRowMapper(rowMapper); + if (fetchSize != null) { + ret = ret.withFetchSize(fetchSize); + } + return ret; } public static Builder builder() { @@ -234,6 +266,7 @@ public static Builder builder() { .setCountQueryTimeoutMillis(SPLITTER_DEFAULT_COUNT_QUERY_TIMEOUT_MILLIS) .setDbParallelizationForSplitProcess(null) .setDbParallelizationForReads(null) + .setFetchSize(null) .setAutoAdjustMaxPartitions(true); } @@ -451,6 +484,8 @@ public abstract Builder setDataSourceProviderFn( public abstract Builder setRowMapper(JdbcIO.RowMapper value); + public abstract Builder setFetchSize(@Nullable Integer value); + public abstract Builder setAdditionalOperationsOnRanges( @Nullable PTransform>, ?> value); diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/PipelineController.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/PipelineController.java index 637fa0ae23..69277ae89d 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/PipelineController.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/templates/PipelineController.java @@ -294,7 +294,8 @@ public JdbcIOWrapperConfig getJDBCIOWrapperConfig( options.getJdbcDriverJars(), options.getMaxConnections(), options.getNumPartitions(), - waitOnSignal); + waitOnSignal, + options.getFetchSize()); } @Override diff --git a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java index 7210706a0b..9ea7010288 100644 --- a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java +++ b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java @@ -66,6 +66,16 @@ public void testConfigWithMySqlDefaultsFromOptions() { assertThat(config.dbAuth().getUserName().get()).isEqualTo(testUser); assertThat(config.dbAuth().getPassword().get()).isEqualTo(testPassword); assertThat(config.waitOn()).isNotNull(); + assertThat(config.maxFetchSize()).isNull(); + sourceDbToSpannerOptions.setFetchSize(42); + assertThat( + OptionsToConfigBuilder.getJdbcIOWrapperConfigWithDefaults( + sourceDbToSpannerOptions, + List.of("table1", "table2"), + null, + Wait.on(dummyPCollection)) + .maxFetchSize()) + .isEqualTo(42); } @Test @@ -89,7 +99,8 @@ public void testConfigWithMySqlUrlFromOptions() { "mysql-jar", 10, 0, - Wait.on(dummyPCollection)); + Wait.on(dummyPCollection), + null); JdbcIOWrapperConfig configWithoutConnectionProperties = OptionsToConfigBuilder.getJdbcIOWrapperConfig( @@ -108,7 +119,8 @@ public void testConfigWithMySqlUrlFromOptions() { "mysql-jar", 10, 0, - Wait.on(dummyPCollection)); + Wait.on(dummyPCollection), + null); assertThat(configWithConnectionProperties.sourceDbURL()) .isEqualTo( @@ -172,7 +184,8 @@ public void testConfigWithPostgreSqlUrlFromOptions() { "mysql-jar", 10, 0, - Wait.on(dummyPCollection)); + Wait.on(dummyPCollection), + null); JdbcIOWrapperConfig configWithoutConnectionParameters = OptionsToConfigBuilder.getJdbcIOWrapperConfig( SQLDialect.POSTGRESQL, @@ -190,7 +203,8 @@ public void testConfigWithPostgreSqlUrlFromOptions() { "mysql-jar", 10, 0, - Wait.on(dummyPCollection)); + Wait.on(dummyPCollection), + null); assertThat(configWithoutConnectionParameters.sourceDbURL()) .isEqualTo("jdbc:postgresql://myhost:5432/mydb?currentSchema=public"); assertThat(configWithConnectionParameters.sourceDbURL()) @@ -218,7 +232,8 @@ public void testConfigWithPostgreSqlUrlWithNamespace() { "mysql-jar", 10, 0, - Wait.on(dummyPCollection)); + Wait.on(dummyPCollection), + null); assertThat(configWithNamespace.sourceDbURL()) .isEqualTo("jdbc:postgresql://myhost:5432/mydb?currentSchema=mynamespace"); } diff --git a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapperTest.java b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapperTest.java index 1c8335b4de..da129dc20f 100644 --- a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapperTest.java +++ b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/iowrapper/JdbcIoWrapperTest.java @@ -372,6 +372,16 @@ public void testReadWithUniformPartitionFeatureFlag() throws RetriableSchemaDisc assertThat( jdbcIOWrapperWithFeatureEnabled.getTableReaders().values().stream().findFirst().get()) .isInstanceOf(ReadWithUniformPartitions.class); + // We test that setting the fetch size works for both modes. The more detailed testing of the + // fetch size getting applied to JdbcIO is covered in {@link ReadWithUniformPartitionTest} + assertThat( + JdbcIoWrapper.of(configWithFeatureEnabled.toBuilder().setMaxFetchSize(42).build()) + .getTableReaders()) + .hasSize(1); + assertThat( + JdbcIoWrapper.of(configWithFeatureDisabled.toBuilder().setMaxFetchSize(42).build()) + .getTableReaders()) + .hasSize(1); } @Test diff --git a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitionsTest.java b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitionsTest.java index 73d90a1292..9f58aabdee 100644 --- a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitionsTest.java +++ b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/uniformsplitter/transforms/ReadWithUniformPartitionsTest.java @@ -32,6 +32,12 @@ */ import static com.google.common.truth.Truth.assertThat; import static org.junit.Assert.assertThrows; +import static org.mockito.ArgumentMatchers.anyInt; +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.cloud.teleport.v2.source.reader.io.jdbc.dialectadapter.mysql.MysqlDialectAdapter; import com.google.cloud.teleport.v2.source.reader.io.jdbc.dialectadapter.mysql.MysqlDialectAdapter.MySqlVersion; @@ -44,6 +50,7 @@ import java.sql.SQLException; import java.util.Iterator; import javax.sql.DataSource; +import org.apache.beam.sdk.io.jdbc.JdbcIO; import org.apache.beam.sdk.io.jdbc.JdbcIO.RowMapper; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -233,6 +240,48 @@ public void testMaxPartitionAutoInference() { assertThat(readWithUniformPartitionsLargeRowCount.maxPartitionsHint()).isEqualTo(4472L); } + @Test + public void testBuildJdbc() { + JdbcIO.ReadAll mockReadAll = mock(JdbcIO.ReadAll.class); + String testQuery = "Select *"; + JdbcIO.PreparedStatementSetter mockRangePrepareator = + mock(JdbcIO.PreparedStatementSetter.class); + SerializableFunction mockDataSourceProviderFn = + mock(SerializableFunction.class); + JdbcIO.RowMapper mockRowMapper = mock(RowMapper.class); + Integer testFetchSize = 42; + + when(mockReadAll.withQuery(testQuery)).thenReturn(mockReadAll); + when(mockReadAll.withParameterSetter(mockRangePrepareator)).thenReturn(mockReadAll); + when(mockReadAll.withDataSourceProviderFn(mockDataSourceProviderFn)).thenReturn(mockReadAll); + when(mockReadAll.withOutputParallelization(false)).thenReturn(mockReadAll); + when(mockReadAll.withRowMapper(mockRowMapper)).thenReturn(mockReadAll); + when(mockReadAll.withFetchSize(testFetchSize)).thenReturn(mockReadAll); + + ReadWithUniformPartitions.buildJdbcIO( + mockReadAll, + testQuery, + mockRangePrepareator, + mockDataSourceProviderFn, + mockRowMapper, + null); + // No fetch size set. + verify(mockReadAll, never()).withFetchSize(anyInt()); + ReadWithUniformPartitions.buildJdbcIO( + mockReadAll, + testQuery, + mockRangePrepareator, + mockDataSourceProviderFn, + mockRowMapper, + testFetchSize); + verify(mockReadAll, times(1)).withFetchSize(testFetchSize); + verify(mockReadAll, times(2)).withQuery(testQuery); + verify(mockReadAll, times(2)).withParameterSetter(mockRangePrepareator); + verify(mockReadAll, times(2)).withDataSourceProviderFn(mockDataSourceProviderFn); + verify(mockReadAll, times(2)).withOutputParallelization(false); + verify(mockReadAll, times(2)).withRowMapper(mockRowMapper); + } + @Test public void testMaxPartitionAutoInferencePreConditions() { Range initialRangeWithWrongColumChild = From 3455819e67347de7f78210f98e13d3ec11211481 Mon Sep 17 00:00:00 2001 From: Manit Gupta Date: Thu, 28 Nov 2024 16:13:27 +0530 Subject: [PATCH 03/14] feat: Modularization: Terraform changes for schema overrides (#1841) * Terraform changes for schema overrides * Fix bucket * Add schema overrides sample json --- .../samples/mysql-end-to-end/README.md | 53 +++++++++++++++- .../samples/mysql-end-to-end/main.tf | 16 ++++- .../mysql-end-to-end/schema-overrides.json | 13 ++++ .../samples/mysql-end-to-end/terraform.tfvars | 3 + .../samples/mysql-end-to-end/variables.tf | 3 + .../mysql-sharded-end-to-end/README.md | 62 +++++++++++++++++-- .../samples/mysql-sharded-end-to-end/main.tf | 13 ++++ .../mysql-sharded-end-to-end/terraform.tfvars | 3 + .../mysql-sharded-end-to-end/variables.tf | 3 + .../samples/postgresql-end-to-end/README.md | 53 +++++++++++++++- .../samples/postgresql-end-to-end/main.tf | 13 ++++ .../postgresql-end-to-end/terraform.tfvars | 4 ++ .../postgresql-end-to-end/variables.tf | 3 + .../pre-configured-conn-profiles/README.md | 53 +++++++++++++++- .../pre-configured-conn-profiles/main.tf | 13 ++++ .../terraform.tfvars | 4 ++ .../pre-configured-conn-profiles/variables.tf | 3 + 17 files changed, 305 insertions(+), 10 deletions(-) create mode 100644 v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/schema-overrides.json diff --git a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/README.md b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/README.md index 1527538ee2..81a64f1d93 100644 --- a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/README.md +++ b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/README.md @@ -453,7 +453,58 @@ mysql_database = { By default, the Dataflow job performs a like-like mapping between source and Spanner. Any schema changes between source and Spanner can be -specified using the `session file`. To specify a session file - +specified in multiple ways - + +#### Using string overrides + +Table and Column name overrides can be specified using +the `var.dataflow_params.template_params.tableOverrides` +and the `var.dataflow_params.template_params.columnOverrides` parameters. + +1. Table overrides are written in the following + format: `[{SourceTableName1, SpannerTableName1}, {SourceTableName2, SpannerTableName2}]` + + For example - `[{Singers, Vocalists}, {Albums, Records}]` + This example shows mapping Singers table to Vocalists and Albums + table to Records. +2. Column overrides are written in the following + format: `[{SourceTableName1.SourceColumnName1, SourceTableName1.SpannerColumnName1}, {SourceTableName2.SourceColumnName1, SourceTableName2.SpannerColumnName1}]`. + Note that the SourceTableName should remain the same in both the source and + spanner pair. To override table names, use tableOverrides. + For + example - `[{Singers.SingerName, Singers.TalentName}, {Albums.AlbumName, Albums.RecordName}]` + The example shows mapping SingerName to TalentName and AlbumName to + RecordName in Singers and Albums table respectively. + +#### Using file based overrides + +You can also use a file to specify the list of overrides. This file can be +placed in a local directory and its path can be configured in the +`var.dataflow_params.template_params.local_schema_overrides_file_path`. The file +will be automatically uploaded to GCS and configured in the template. + +A sample override file is - + +```json +{ + "renamedTables":{ + "srcTable":"destTable" + }, + "renamedColumns":{ + "srcTable1":{ + "srcCol1":"destCol1" + }, + "srcTable2":{ + "srcCol2":"destCol2" + } + } +} +``` + +#### Using session file + +The session file is generated via Spanner Migration Tool. To specify a session +file - 1. Copy the contents of the SMT generated `session file` to the `session.json` file. diff --git a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/main.tf b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/main.tf index 27c6535f94..cf590e5266 100644 --- a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/main.tf +++ b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/main.tf @@ -102,6 +102,16 @@ resource "google_storage_bucket_object" "session_file_object" { bucket = google_storage_bucket.datastream_bucket.id } +# upload local schema overrides file to the created GCS bucket +resource "google_storage_bucket_object" "schema_overrides_file_object" { + count = var.dataflow_params.template_params.local_schema_overrides_file_path != null ? 1 : 0 + depends_on = [google_project_service.enabled_apis] + name = "schema-overrides.json" + source = var.dataflow_params.template_params.local_schema_overrides_file_path + content_type = "application/json" + bucket = google_storage_bucket.datastream_bucket.id +} + # Pub/Sub Topic for Datastream resource "google_pubsub_topic" "datastream_topic" { depends_on = [google_project_service.enabled_apis] @@ -244,7 +254,8 @@ resource "google_project_iam_member" "live_migration_roles" { resource "google_dataflow_flex_template_job" "live_migration_job" { count = var.dataflow_params.skip_dataflow ? 0 : 1 depends_on = [ - google_project_service.enabled_apis, google_project_iam_member.live_migration_roles + google_project_service.enabled_apis, + google_project_iam_member.live_migration_roles ] # Launch the template once the stream is created. provider = google-beta container_spec_gcs_path = "gs://dataflow-templates-${var.common_params.region}/latest/flex/Cloud_Datastream_to_Spanner" @@ -278,6 +289,9 @@ resource "google_dataflow_flex_template_job" "live_migration_job" { transformationClassName = var.dataflow_params.template_params.transformation_class_name transformationCustomParameters = var.dataflow_params.template_params.transformation_custom_parameters filteredEventsDirectory = var.dataflow_params.template_params.filtered_events_directory + tableOverrides = var.dataflow_params.template_params.table_overrides + columnOverrides = var.dataflow_params.template_params.column_overrides + schemaOverridesFilePath = var.dataflow_params.template_params.local_schema_overrides_file_path != null ? "gs://${google_storage_bucket_object.schema_overrides_file_object[0].bucket}/${google_storage_bucket_object.schema_overrides_file_object[0].name}" : null } # Additional Job Configurations diff --git a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/schema-overrides.json b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/schema-overrides.json new file mode 100644 index 0000000000..f6a10824b2 --- /dev/null +++ b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/schema-overrides.json @@ -0,0 +1,13 @@ +{ + "renamedTables":{ + "srcTable":"destTable" + }, + "renamedColumns":{ + "srcTable1":{ + "srcCol1":"destCol1" + }, + "srcTable2":{ + "srcCol2":"destCol2" + } + } +} \ No newline at end of file diff --git a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/terraform.tfvars b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/terraform.tfvars index 2c5df5570d..62ba5a82bd 100644 --- a/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/terraform.tfvars +++ b/v2/datastream-to-spanner/terraform/samples/mysql-end-to-end/terraform.tfvars @@ -74,6 +74,9 @@ dataflow_params = { transformation_custom_parameters = "" # Optional transformation_class_name = "" # Fully Classified Class Name(Optional) filtered_events_directory = "" # Optional + table_overrides = " Date: Fri, 29 Nov 2024 15:41:28 +0000 Subject: [PATCH 04/14] big-int-unsgined transforms (#2039) --- .../provider/MysqlJdbcValueMappings.java | 6 +- .../provider/MysqlMappingProvider.java | 2 +- .../rowmapper/JdbcSourceRowMapperTest.java | 2 +- .../provider/MysqlMappingProviderTest.java | 4 +- .../avro/GenericRecordTypeConvertor.java | 12 +- .../avro/GenericRecordTypeConvertorTest.java | 114 +++++++++++++++++- 6 files changed, 129 insertions(+), 11 deletions(-) diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/provider/MysqlJdbcValueMappings.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/provider/MysqlJdbcValueMappings.java index d501db5978..43f23afdee 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/provider/MysqlJdbcValueMappings.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/provider/MysqlJdbcValueMappings.java @@ -66,6 +66,10 @@ public class MysqlJdbcValueMappings implements JdbcValueMappingsProvider { .unscaledValue() .toByteArray()); + /** Map BigInt Unsigned type to Avro Number. */ + private static final ResultSetValueMapper bigDecimalToAvroNumber = + (value, schema) -> value.toString(); + /* Hex Encoded string for bytes type. */ private static final ResultSetValueMapper bytesToHexString = (value, schema) -> new String(Hex.encodeHex(value)); @@ -191,7 +195,7 @@ private static long instantToMicro(Instant instant) { private static final ImmutableMap> SCHEMA_MAPPINGS = ImmutableMap., ResultSetValueMapper>>builder() .put("BIGINT", Pair.of(ResultSet::getLong, valuePassThrough)) - .put("BIGINT UNSIGNED", Pair.of(ResultSet::getBigDecimal, bigDecimalToByteArray)) + .put("BIGINT UNSIGNED", Pair.of(ResultSet::getBigDecimal, bigDecimalToAvroNumber)) .put("BINARY", Pair.of(ResultSet::getBytes, bytesToHexString)) .put("BIT", Pair.of(ResultSet::getBytes, bytesToLong)) .put("BLOB", Pair.of(ResultSet::getBlob, blobToHexString)) diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProvider.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProvider.java index 536c8a3fcf..8ea3ee32b4 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProvider.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProvider.java @@ -33,7 +33,7 @@ public final class MysqlMappingProvider { private static final ImmutableMap MAPPING = ImmutableMap.builder() .put("BIGINT", UnifiedMappingProvider.Type.LONG) - .put("BIGINT UNSIGNED", UnifiedMappingProvider.Type.DECIMAL) + .put("BIGINT UNSIGNED", UnifiedMappingProvider.Type.NUMBER) .put("BINARY", UnifiedMappingProvider.Type.STRING) .put("BIT", UnifiedMappingProvider.Type.LONG) .put("BLOB", UnifiedMappingProvider.Type.STRING) diff --git a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/JdbcSourceRowMapperTest.java b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/JdbcSourceRowMapperTest.java index d1fe943d38..df0abfe098 100644 --- a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/JdbcSourceRowMapperTest.java +++ b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/jdbc/rowmapper/JdbcSourceRowMapperTest.java @@ -272,7 +272,7 @@ private static ImmutableList mySQLColumns() { .derbyColumnType("BIGINT") .sourceColumnType("BIGINT UNSIGNED", new Long[] {20L, 0L}) .inputValue(12345L) - .mappedValue(ByteBuffer.wrap(new byte[] {(byte) 0x30, (byte) 0x39})) + .mappedValue("12345") .build()) .add( Column.builder() diff --git a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProviderTest.java b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProviderTest.java index b32ef0c6eb..56e883450a 100644 --- a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProviderTest.java +++ b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/source/reader/io/schema/typemapping/provider/MysqlMappingProviderTest.java @@ -49,9 +49,7 @@ public void testMySqlMappingProvider() { private ImmutableMap expectedMapping() { return ImmutableMap.builder() .put("BIGINT", "\"long\"") - .put( - "BIGINT UNSIGNED", - "{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":1,\"scale\":1}") + .put("BIGINT UNSIGNED", "{\"type\":\"string\",\"logicalType\":\"number\"}") .put("BINARY", "\"string\"") .put("BIT", "\"long\"") .put("BLOB", "\"string\"") diff --git a/v2/spanner-common/src/main/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertor.java b/v2/spanner-common/src/main/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertor.java index e00bcfe2ca..4c7ea2cdc6 100644 --- a/v2/spanner-common/src/main/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertor.java +++ b/v2/spanner-common/src/main/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertor.java @@ -47,6 +47,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.kerby.util.Hex; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -252,15 +253,20 @@ private Map genericRecordToMap(GenericRecord record) { switch (fieldType) { case INT: case LONG: + fieldValue = Long.valueOf(fieldValue.toString()); + break; case BOOLEAN: - fieldValue = (fieldValue == null) ? null : Long.valueOf(fieldValue.toString()); + fieldValue = Boolean.valueOf(fieldValue.toString()); break; case FLOAT: case DOUBLE: - fieldValue = (fieldValue == null) ? null : Double.valueOf(fieldValue.toString()); + fieldValue = Double.valueOf(fieldValue.toString()); + break; + case BYTES: + fieldValue = Hex.encode(((ByteBuffer) fieldValue).array()); break; default: - fieldValue = (fieldValue == null) ? null : fieldValue.toString(); + fieldValue = fieldValue.toString(); } map.put(fieldName, fieldValue); } diff --git a/v2/spanner-common/src/test/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertorTest.java b/v2/spanner-common/src/test/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertorTest.java index 43f2a068ad..68b94f69d3 100644 --- a/v2/spanner-common/src/test/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertorTest.java +++ b/v2/spanner-common/src/test/java/com/google/cloud/teleport/v2/spanner/migrations/avro/GenericRecordTypeConvertorTest.java @@ -33,6 +33,9 @@ import com.google.cloud.teleport.v2.spanner.migrations.schema.IdentityMapper; import com.google.cloud.teleport.v2.spanner.migrations.schema.SessionBasedMapper; import com.google.cloud.teleport.v2.spanner.type.Type; +import com.google.cloud.teleport.v2.spanner.utils.ISpannerMigrationTransformer; +import com.google.cloud.teleport.v2.spanner.utils.MigrationTransformationRequest; +import com.google.cloud.teleport.v2.spanner.utils.MigrationTransformationResponse; import com.google.common.io.Resources; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -46,6 +49,7 @@ import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.collections.map.HashedMap; import org.junit.Test; import org.mockito.Mockito; @@ -393,9 +397,14 @@ public void transformChangeEventTest_identityMapper() throws InvalidTransformati genericRecord.put("bytes_col", ByteBuffer.wrap(new byte[] {10, 20, 30})); genericRecord.put("timestamp_col", 1602599400056483L); genericRecord.put("date_col", 738991); + + GenericRecord genericRecordAllNulls = new GenericData.Record(getAllSpannerTypesSchema()); + getAllSpannerTypesSchema().getFields().stream() + .forEach(f -> genericRecordAllNulls.put(f.name(), null)); + GenericRecordTypeConvertor genericRecordTypeConvertor = new GenericRecordTypeConvertor(new IdentityMapper(getIdentityDdl()), "", null, null); - Map actual = + Map actualWithoutCustomTransform = genericRecordTypeConvertor.transformChangeEvent(genericRecord, "all_types"); Map expected = Map.of( @@ -408,7 +417,72 @@ public void transformChangeEventTest_identityMapper() throws InvalidTransformati "timestamp_col", Value.timestamp(Timestamp.parseTimestamp("2020-10-13T14:30:00.056483Z")), "date_col", Value.date(com.google.cloud.Date.parseDate("3993-04-16"))); - assertEquals(expected, actual); + // Implementation Detail, the transform returns Spanner values, and Value.Null is not equal to + // java null, + // So simple transform for expected map to have null values does not work for us. + Map expectedNulls = + Map.of( + "bool_col", + Value.bool(null), + "int_col", + Value.int64(null), + "float_col", + Value.float64(null), + "string_col", + Value.string(null), + "numeric_col", + Value.numeric(null), + "bytes_col", + Value.bytes(null), + "timestamp_col", + Value.timestamp(null), + "date_col", + Value.date(null)); + Map actualWithCustomTransform = + new GenericRecordTypeConvertor( + new IdentityMapper(getIdentityDdl()), + "", + null, + new TestCustomTransform(expected, false, false)) + .transformChangeEvent(genericRecord, "all_types"); + + /* Checks that when there's no custom transform, output is as expected */ + assertEquals(expected, actualWithoutCustomTransform); + + /* Checks for the part of the code that supplies inputs to custom transforms */ + + /* Check correct input map generated when using customTransform */ + assertEquals(expected, actualWithCustomTransform); + + /* Checks that if any fields is made null by the custom transform, we get output with values as Value.NULL */ + assertEquals( + expectedNulls, + new GenericRecordTypeConvertor( + new IdentityMapper(getIdentityDdl()), + "", + null, + new TestCustomTransform(expected, false, true)) + .transformChangeEvent(genericRecord, "all_types")); + + /* Checks that if event is filtered by the custom transform, output is null. */ + assertEquals( + null, + new GenericRecordTypeConvertor( + new IdentityMapper(getIdentityDdl()), + "", + null, + new TestCustomTransform(expected, true, false)) + .transformChangeEvent(genericRecord, "all_types")); + + /* Checks that if any field in generic record is null, we get custom transform input map entry with value as Value.NULL */ + assertEquals( + expectedNulls, + new GenericRecordTypeConvertor( + new IdentityMapper(getIdentityDdl()), + "", + null, + new TestCustomTransform(expected, false, false)) + .transformChangeEvent(genericRecordAllNulls, "all_types")); } @Test @@ -796,4 +870,40 @@ public void transformChangeEventTest_SynthPKPopulation() throws InvalidTransform assertTrue(actual.containsKey("synth_id")); assertEquals(Value.string("name1"), actual.get("new_name")); } + + private class TestCustomTransform implements ISpannerMigrationTransformer { + + private Map expected; + private Boolean isFiltered; + private Boolean nullify; + + public TestCustomTransform(Map expected, boolean isFiltered, boolean nullify) { + this.expected = expected; + this.isFiltered = isFiltered; + this.nullify = nullify; + } + + @Override + public void init(String customParameters) {} + + @Override + public MigrationTransformationResponse toSpannerRow(MigrationTransformationRequest request) + throws InvalidTransformationException { + if (!nullify) { + return new MigrationTransformationResponse(request.getRequestRow(), isFiltered); + } else { + Map allNulls = new HashedMap(); + for (String k : request.getRequestRow().keySet()) { + allNulls.put(k, null); + } + return new MigrationTransformationResponse(allNulls, isFiltered); + } + } + + @Override + public MigrationTransformationResponse toSourceRow(MigrationTransformationRequest request) + throws InvalidTransformationException { + return null; + } + } } From 9854a705d1ef07f779c0aface8eada47c0d7676b Mon Sep 17 00:00:00 2001 From: Sailesh Mukil Date: Fri, 29 Nov 2024 09:36:08 -0800 Subject: [PATCH 05/14] Add support for PropertyGraphs in import/export DDL (#2003) * Add support for PropertyGraphs in import/export DDL * Apply mvn spotless * PropertyGraph DDL-Avro converters * Graph information schema * Graph import/export * Minor cleanups and mvn:spotless * Address review comments and re-enable ImportFromAvroTest#propertyGraphs --------- Co-authored-by: Sailesh Mukil Gangatharan --- .../spanner/AvroSchemaToDdlConverter.java | 244 ++++++++++++++++ .../cloud/teleport/spanner/AvroUtil.java | 7 + .../spanner/DdlToAvroSchemaConverter.java | 122 ++++++++ .../teleport/spanner/ExportTransform.java | 35 +++ .../teleport/spanner/ImportTransform.java | 26 +- .../cloud/teleport/spanner/ddl/Ddl.java | 54 ++++ .../spanner/ddl/GraphElementTable.java | 246 ++++++++++++++++ .../spanner/ddl/InformationSchemaScanner.java | 276 ++++++++++++++++++ .../teleport/spanner/ddl/PropertyGraph.java | 234 +++++++++++++++ .../spanner/AvroSchemaToDdlConverterTest.java | 66 +++++ .../spanner/DdlToAvroSchemaConverterTest.java | 159 ++++++++++ .../teleport/spanner/ExportTransformTest.java | 12 +- .../teleport/spanner/ImportFromAvroTest.java | 86 ++++++ .../cloud/teleport/spanner/ddl/DdlTest.java | 95 ++++++ .../spanner/ddl/GraphElementTableTest.java | 136 +++++++++ .../ddl/InformationSchemaScannerIT.java | 112 +++++++ .../spanner/ddl/PropertyGraphTest.java | 205 +++++++++++++ 17 files changed, 2107 insertions(+), 8 deletions(-) create mode 100644 v1/src/main/java/com/google/cloud/teleport/spanner/ddl/GraphElementTable.java create mode 100644 v1/src/main/java/com/google/cloud/teleport/spanner/ddl/PropertyGraph.java create mode 100644 v1/src/test/java/com/google/cloud/teleport/spanner/ddl/GraphElementTableTest.java create mode 100644 v1/src/test/java/com/google/cloud/teleport/spanner/ddl/PropertyGraphTest.java diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java b/v1/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java index 0fb0f6a89b..4047d86f90 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverter.java @@ -23,17 +23,22 @@ import static com.google.cloud.teleport.spanner.AvroUtil.OUTPUT; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_CHANGE_STREAM_FOR_CLAUSE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_CHECK_CONSTRAINT; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_EDGE_TABLE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_MODEL; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_PLACEMENT; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_PROPERTY_GRAPH; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_FOREIGN_KEY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_INDEX; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_LABEL; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_NAMED_SCHEMA; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_NODE_TABLE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ON_DELETE_ACTION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_OPTION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PARENT; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PLACEMENT_KEY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PRIMARY_KEY; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PROPERTY_DECLARATION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_REMOTE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_SEQUENCE_COUNTER_START; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_SEQUENCE_KIND; @@ -53,9 +58,13 @@ 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.GraphElementTable; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.Kind; import com.google.cloud.teleport.spanner.ddl.Model; import com.google.cloud.teleport.spanner.ddl.NamedSchema; import com.google.cloud.teleport.spanner.ddl.Placement; +import com.google.cloud.teleport.spanner.ddl.PropertyGraph; +import com.google.cloud.teleport.spanner.ddl.PropertyGraph.PropertyDeclaration; import com.google.cloud.teleport.spanner.ddl.Sequence; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.ddl.View; @@ -102,6 +111,8 @@ public Ddl toDdl(Collection avroSchemas) { builder.addSchema(toSchema(null, schema)); } else if (SPANNER_ENTITY_PLACEMENT.equals(schema.getProp(SPANNER_ENTITY))) { builder.addPlacement(toPlacement(null, schema)); + } else if (SPANNER_ENTITY_PROPERTY_GRAPH.equals(schema.getProp(SPANNER_ENTITY))) { + builder.addPropertyGraph(toPropertyGraph(null, schema)); } else { builder.addTable(toTable(null, schema)); } @@ -131,6 +142,239 @@ public View toView(String viewName, Schema schema) { return builder.build(); } + // TODO: Modularize long function implementation using helpers + public PropertyGraph toPropertyGraph(String propertyGraphName, Schema schema) { + PropertyGraph.Builder propertyGraphBuilder = PropertyGraph.builder(); + if (propertyGraphName == null) { + propertyGraphName = getSpannerObjectName(schema); + } + LOG.debug("Converting to Ddl propertyGraphName {}", propertyGraphName); + + propertyGraphBuilder.name(propertyGraphName); + + // Deserialize nodeTables + int nodeTableCount = 0; + while (schema.getProp(SPANNER_NODE_TABLE + "_" + nodeTableCount + "_NAME") != null) { + GraphElementTable.Builder nodeTableBuilder = GraphElementTable.builder(); + nodeTableBuilder.name(schema.getProp(SPANNER_NODE_TABLE + "_" + nodeTableCount + "_NAME")); + nodeTableBuilder.baseTableName( + schema.getProp(SPANNER_NODE_TABLE + "_" + nodeTableCount + "_BASE_TABLE_NAME")); + nodeTableBuilder.kind(Kind.NODE); + + // Deserialize keyColumns + String[] keyColumns = + schema + .getProp(SPANNER_NODE_TABLE + "_" + nodeTableCount + "_KEY_COLUMNS") + .trim() + .split(","); + nodeTableBuilder.keyColumns(ImmutableList.copyOf(keyColumns)); + + // Deserialize labelToPropertyDefinitions + int labelCount = 0; + ImmutableList.Builder labelsBuilder = + ImmutableList.builder(); + while (schema.getProp( + SPANNER_NODE_TABLE + "_" + nodeTableCount + "_LABEL_" + labelCount + "_NAME") + != null) { + String labelName = + schema.getProp( + SPANNER_NODE_TABLE + "_" + nodeTableCount + "_LABEL_" + labelCount + "_NAME"); + ImmutableList.Builder propertyDefinitionsBuilder = + ImmutableList.builder(); + int propertyCount = 0; + while (schema.getProp( + SPANNER_NODE_TABLE + + "_" + + nodeTableCount + + "_LABEL_" + + labelCount + + "_PROPERTY_" + + propertyCount + + "_NAME") + != null) { + String propertyName = + schema.getProp( + SPANNER_NODE_TABLE + + "_" + + nodeTableCount + + "_LABEL_" + + labelCount + + "_PROPERTY_" + + propertyCount + + "_NAME"); + String propertyValue = + schema.getProp( + SPANNER_NODE_TABLE + + "_" + + nodeTableCount + + "_LABEL_" + + labelCount + + "_PROPERTY_" + + propertyCount + + "_VALUE"); + propertyDefinitionsBuilder.add( + new GraphElementTable.PropertyDefinition(propertyName, propertyValue)); + propertyCount++; + } + labelsBuilder.add( + new GraphElementTable.LabelToPropertyDefinitions( + labelName, propertyDefinitionsBuilder.build())); + labelCount++; + } + nodeTableBuilder.labelToPropertyDefinitions(labelsBuilder.build()); + + propertyGraphBuilder.addNodeTable(nodeTableBuilder.autoBuild()); + nodeTableCount++; + } + + // Deserialize edgeTables (similar logic to nodeTables) + int edgeTableCount = 0; + while (schema.getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_NAME") != null) { + GraphElementTable.Builder edgeTableBuilder = GraphElementTable.builder(); + edgeTableBuilder.name(schema.getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_NAME")); + edgeTableBuilder.baseTableName( + schema.getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_BASE_TABLE_NAME")); + edgeTableBuilder.kind(Kind.EDGE); + + // Deserialize keyColumns + String[] keyColumns = + schema + .getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_KEY_COLUMNS") + .trim() + .split(","); + edgeTableBuilder.keyColumns(ImmutableList.copyOf(keyColumns)); + + // Deserialize labelToPropertyDefinitions + int labelCount = 0; + ImmutableList.Builder labelsBuilder = + ImmutableList.builder(); + while (schema.getProp( + SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_LABEL_" + labelCount + "_NAME") + != null) { + String labelName = + schema.getProp( + SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_LABEL_" + labelCount + "_NAME"); + ImmutableList.Builder propertyDefinitionsBuilder = + ImmutableList.builder(); + int propertyCount = 0; + while (schema.getProp( + SPANNER_EDGE_TABLE + + "_" + + edgeTableCount + + "_LABEL_" + + labelCount + + "_PROPERTY_" + + propertyCount + + "_NAME") + != null) { + String propertyName = + schema.getProp( + SPANNER_EDGE_TABLE + + "_" + + edgeTableCount + + "_LABEL_" + + labelCount + + "_PROPERTY_" + + propertyCount + + "_NAME"); + String propertyValue = + schema.getProp( + SPANNER_EDGE_TABLE + + "_" + + edgeTableCount + + "_LABEL_" + + labelCount + + "_PROPERTY_" + + propertyCount + + "_VALUE"); + propertyDefinitionsBuilder.add( + new GraphElementTable.PropertyDefinition(propertyName, propertyValue)); + propertyCount++; + } + labelsBuilder.add( + new GraphElementTable.LabelToPropertyDefinitions( + labelName, propertyDefinitionsBuilder.build())); + labelCount++; + } + edgeTableBuilder.labelToPropertyDefinitions(labelsBuilder.build()); + + // Deserialize sourceNodeTable and targetNodeTable (always present for edges) + String sourceNodeTableName = + schema.getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_SOURCE_NODE_TABLE_NAME"); + String[] sourceNodeKeyColumns = + schema + .getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_SOURCE_NODE_KEY_COLUMNS") + .trim() + .split(","); + String[] sourceEdgeKeyColumns = + schema + .getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_SOURCE_EDGE_KEY_COLUMNS") + .trim() + .split(","); + edgeTableBuilder.sourceNodeTable( + new GraphElementTable.GraphNodeTableReference( + sourceNodeTableName, + ImmutableList.copyOf(sourceNodeKeyColumns), + ImmutableList.copyOf(sourceEdgeKeyColumns))); + + String targetNodeTableName = + schema.getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_TARGET_NODE_TABLE_NAME"); + String[] targetNodeKeyColumns = + schema + .getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_TARGET_NODE_KEY_COLUMNS") + .trim() + .split(","); + String[] targetEdgeKeyColumns = + schema + .getProp(SPANNER_EDGE_TABLE + "_" + edgeTableCount + "_TARGET_EDGE_KEY_COLUMNS") + .trim() + .split(","); + edgeTableBuilder.targetNodeTable( + new GraphElementTable.GraphNodeTableReference( + targetNodeTableName, + ImmutableList.copyOf(targetNodeKeyColumns), + ImmutableList.copyOf(targetEdgeKeyColumns))); + + propertyGraphBuilder.addEdgeTable(edgeTableBuilder.autoBuild()); + edgeTableCount++; + } + + // Deserialize propertyDeclarations + int propertyDeclCount = 0; + ImmutableList.Builder propertyDeclsBuilder = + ImmutableList.builder(); + while (schema.getProp(SPANNER_PROPERTY_DECLARATION + "_" + propertyDeclCount + "_NAME") + != null) { + String propertyName = + schema.getProp(SPANNER_PROPERTY_DECLARATION + "_" + propertyDeclCount + "_NAME"); + String propertyType = + schema.getProp(SPANNER_PROPERTY_DECLARATION + "_" + propertyDeclCount + "_TYPE"); + propertyGraphBuilder.addPropertyDeclaration( + new PropertyDeclaration(propertyName, propertyType)); + propertyDeclCount++; + } + + // Deserialize labels + int labelCount = 0; + ImmutableList.Builder labelsBuilder = ImmutableList.builder(); + while (schema.getProp(SPANNER_LABEL + "_" + labelCount + "_NAME") != null) { + String labelName = schema.getProp(SPANNER_LABEL + "_" + labelCount + "_NAME"); + ImmutableList.Builder labelPropertiesBuilder = ImmutableList.builder(); + int propertyCount = 0; + while (schema.getProp(SPANNER_LABEL + "_" + labelCount + "_PROPERTY_" + propertyCount) + != null) { + labelPropertiesBuilder.add( + schema.getProp(SPANNER_LABEL + "_" + labelCount + "_PROPERTY_" + propertyCount)); + propertyCount++; + } + propertyGraphBuilder.addLabel( + new PropertyGraph.GraphElementLabel(labelName, labelPropertiesBuilder.build())); + labelCount++; + } + + return propertyGraphBuilder.build(); + } + public Model toModel(String modelName, Schema schema) { if (modelName == null) { modelName = getSpannerObjectName(schema); diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java b/v1/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java index 8ecfb505d4..544b3debf4 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/AvroUtil.java @@ -35,6 +35,7 @@ private AvroUtil() {} public static final String SPANNER_CHANGE_STREAM_FOR_CLAUSE = "spannerChangeStreamForClause"; public static final String SPANNER_ENTITY = "spannerEntity"; public static final String SPANNER_ENTITY_MODEL = "Model"; + public static final String SPANNER_ENTITY_PROPERTY_GRAPH = "PropertyGraph"; public static final String SPANNER_ENTITY_PLACEMENT = "Placement"; public static final String SPANNER_FOREIGN_KEY = "spannerForeignKey_"; public static final String SPANNER_INDEX = "spannerIndex_"; @@ -51,6 +52,12 @@ private AvroUtil() {} public static final String SPANNER_VIEW_QUERY = "spannerViewQuery"; public static final String SPANNER_VIEW_SECURITY = "spannerViewSecurity"; public static final String SPANNER_NAMED_SCHEMA = "spannerNamedSchema"; + + public static final String SPANNER_NODE_TABLE = "spannerGraphNodeTable"; + public static final String SPANNER_EDGE_TABLE = "spannerGraphEdgeTable"; + public static final String SPANNER_PROPERTY_DECLARATION = "spannerGraphPropertyDeclaration"; + public static final String SPANNER_LABEL = "spannerGraphLabel"; + public static final String SPANNER_NAME = "spannerName"; public static final String STORED = "stored"; public static final String SPANNER_PLACEMENT_KEY = "spannerPlacementKey"; diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java b/v1/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java index 8bf2e59af1..c322c470fa 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverter.java @@ -25,18 +25,23 @@ import static com.google.cloud.teleport.spanner.AvroUtil.OUTPUT; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_CHANGE_STREAM_FOR_CLAUSE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_CHECK_CONSTRAINT; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_EDGE_TABLE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_MODEL; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_PLACEMENT; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_PROPERTY_GRAPH; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_FOREIGN_KEY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_INDEX; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_LABEL; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_NAME; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_NAMED_SCHEMA; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_NODE_TABLE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ON_DELETE_ACTION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_OPTION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PARENT; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PLACEMENT_KEY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PRIMARY_KEY; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PROPERTY_DECLARATION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_REMOTE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_SEQUENCE_COUNTER_START; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_SEQUENCE_KIND; @@ -54,14 +59,17 @@ 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.GraphElementTable; import com.google.cloud.teleport.spanner.ddl.IndexColumn; import com.google.cloud.teleport.spanner.ddl.Model; import com.google.cloud.teleport.spanner.ddl.ModelColumn; import com.google.cloud.teleport.spanner.ddl.NamedSchema; import com.google.cloud.teleport.spanner.ddl.Placement; +import com.google.cloud.teleport.spanner.ddl.PropertyGraph; import com.google.cloud.teleport.spanner.ddl.Sequence; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.ddl.View; +import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.Collection; import java.util.stream.Collectors; @@ -240,6 +248,37 @@ public Collection convert(Ddl ddl) { schemas.add(schema); } + for (PropertyGraph propertyGraph : ddl.propertyGraphs()) { + LOG.info("DdlToAvro PropertyGraph {}", propertyGraph.name()); + SchemaBuilder.RecordBuilder recordBuilder = + SchemaBuilder.record(generateAvroSchemaName(propertyGraph.name())) + .namespace(this.namespace); + + recordBuilder.prop(SPANNER_NAME, propertyGraph.name()); + recordBuilder.prop(GOOGLE_FORMAT_VERSION, version); + recordBuilder.prop(GOOGLE_STORAGE, "CloudSpanner"); + recordBuilder.prop(SPANNER_ENTITY, SPANNER_ENTITY_PROPERTY_GRAPH); + + // Encode nodeTables + for (int i = 0; i < propertyGraph.nodeTables().size(); i++) { + encodeNodeTable(recordBuilder, propertyGraph.nodeTables().get(i), i); + } + + // Encode edgeTables + for (int i = 0; i < propertyGraph.edgeTables().size(); i++) { + encodeEdgeTable(recordBuilder, propertyGraph.edgeTables().get(i), i); + } + + // Encode propertyDeclarations + encodePropertyDeclarations(recordBuilder, propertyGraph.propertyDeclarations()); + + // Encode labels + encodeLabels(recordBuilder, propertyGraph.labels()); + + Schema schema = recordBuilder.fields().endRecord(); + schemas.add(schema); + } + for (View view : ddl.views()) { LOG.info("DdlToAvo view {}", view.name()); SchemaBuilder.RecordBuilder recordBuilder = @@ -321,6 +360,89 @@ public Collection convert(Ddl ddl) { return schemas; } + private void encodeNodeTable( + SchemaBuilder.RecordBuilder recordBuilder, GraphElementTable nodeTable, int i) { + encodeElementTable(recordBuilder, nodeTable, i, SPANNER_NODE_TABLE); + } + + private void encodeEdgeTable( + SchemaBuilder.RecordBuilder recordBuilder, GraphElementTable edgeTable, int i) { + encodeElementTable(recordBuilder, edgeTable, i, SPANNER_EDGE_TABLE); + + // Encode sourceNodeTable and targetNodeTable (always present for edges) + recordBuilder.prop( + SPANNER_EDGE_TABLE + "_" + i + "_SOURCE_NODE_TABLE_NAME", + edgeTable.sourceNodeTable().nodeTableName); + recordBuilder.prop( + SPANNER_EDGE_TABLE + "_" + i + "_SOURCE_NODE_KEY_COLUMNS", + String.join(", ", edgeTable.sourceNodeTable().nodeKeyColumns)); + recordBuilder.prop( + SPANNER_EDGE_TABLE + "_" + i + "_SOURCE_EDGE_KEY_COLUMNS", + String.join(", ", edgeTable.sourceNodeTable().edgeKeyColumns)); + recordBuilder.prop( + SPANNER_EDGE_TABLE + "_" + i + "_TARGET_NODE_TABLE_NAME", + edgeTable.targetNodeTable().nodeTableName); + recordBuilder.prop( + SPANNER_EDGE_TABLE + "_" + i + "_TARGET_NODE_KEY_COLUMNS", + String.join(", ", edgeTable.targetNodeTable().nodeKeyColumns)); + recordBuilder.prop( + SPANNER_EDGE_TABLE + "_" + i + "_TARGET_EDGE_KEY_COLUMNS", + String.join(", ", edgeTable.targetNodeTable().edgeKeyColumns)); + } + + private void encodeElementTable( + SchemaBuilder.RecordBuilder recordBuilder, + GraphElementTable elementTable, + int i, + String prefix) { + recordBuilder.prop(prefix + "_" + i + "_NAME", elementTable.name()); + recordBuilder.prop(prefix + "_" + i + "_BASE_TABLE_NAME", elementTable.baseTableName()); + recordBuilder.prop(prefix + "_" + i + "_KIND", elementTable.kind().toString()); + // Encode keyColumns + recordBuilder.prop( + prefix + "_" + i + "_KEY_COLUMNS", String.join(", ", elementTable.keyColumns())); + + // Encode labelToPropertyDefinitions + for (int j = 0; j < elementTable.labelToPropertyDefinitions().size(); j++) { + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDef = + elementTable.labelToPropertyDefinitions().get(j); + recordBuilder.prop(prefix + "_" + i + "_LABEL_" + j + "_NAME", labelToPropertyDef.labelName); + // Encode propertyDefinitions + for (int k = 0; k < labelToPropertyDef.propertyDefinitions.size(); k++) { + GraphElementTable.PropertyDefinition propertyDef = + labelToPropertyDef.propertyDefinitions.get(k); + recordBuilder.prop( + prefix + "_" + i + "_LABEL_" + j + "_PROPERTY_" + k + "_NAME", propertyDef.name); + recordBuilder.prop( + prefix + "_" + i + "_LABEL_" + j + "_PROPERTY_" + k + "_VALUE", + propertyDef.valueExpressionString); + } + } + } + + private void encodePropertyDeclarations( + SchemaBuilder.RecordBuilder recordBuilder, + ImmutableList declarations) { + for (int i = 0; i < declarations.size(); i++) { + PropertyGraph.PropertyDeclaration declaration = declarations.get(i); + recordBuilder.prop(SPANNER_PROPERTY_DECLARATION + "_" + i + "_NAME", declaration.name); + recordBuilder.prop(SPANNER_PROPERTY_DECLARATION + "_" + i + "_TYPE", declaration.type); + } + } + + private void encodeLabels( + SchemaBuilder.RecordBuilder recordBuilder, + ImmutableList labels) { + for (int i = 0; i < labels.size(); i++) { + PropertyGraph.GraphElementLabel label = labels.get(i); + recordBuilder.prop(SPANNER_LABEL + "_" + i + "_NAME", label.name); + // Encode properties + for (int j = 0; j < label.properties.size(); j++) { + recordBuilder.prop(SPANNER_LABEL + "_" + i + "_PROPERTY_" + j, label.properties.get(j)); + } + } + } + /** * Converts a Spanner type into Avro type. * diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java b/v1/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java index c12154eab9..fae0a377d3 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/ExportTransform.java @@ -26,6 +26,7 @@ import com.google.cloud.teleport.spanner.ddl.Model; import com.google.cloud.teleport.spanner.ddl.NamedSchema; import com.google.cloud.teleport.spanner.ddl.Placement; +import com.google.cloud.teleport.spanner.ddl.PropertyGraph; import com.google.cloud.teleport.spanner.ddl.Sequence; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.proto.ExportProtos; @@ -320,6 +321,21 @@ public void processElement(ProcessContext c) { } })); + PCollection allPropertyGraphNames = + ddl.apply( + "List all property graph names", + ParDo.of( + new DoFn() { + + @ProcessElement + public void processElement(ProcessContext c) { + Ddl ddl = c.element(); + for (PropertyGraph graph : ddl.propertyGraphs()) { + c.output(graph.name()); + } + } + })); + PCollection allChangeStreamNames = ddl.apply( "List all change stream names", @@ -515,6 +531,24 @@ public void processElement(ProcessContext c) { } })); + PCollection>> propertyGraphs = + allPropertyGraphNames.apply( + "Export property graphs", + ParDo.of( + new DoFn>>() { + + @ProcessElement + public void processElement(ProcessContext c) { + String propertyGraphName = c.element(); + LOG.info("Exporting property graph: " + propertyGraphName); + // This file will contain the schema definition for the propertyGraph. + c.output( + KV.of( + propertyGraphName, + Collections.singleton(propertyGraphName + ".avro-00000-of-00001"))); + } + })); + PCollection>> changeStreams = allChangeStreamNames.apply( "Export change streams", @@ -596,6 +630,7 @@ public void processElement(ProcessContext c) { .and(sequences) .and(namedSchemas) .and(placements) + .and(propertyGraphs) .apply("Combine all empty schema files", Flatten.pCollections()); emptySchemaFiles = diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java b/v1/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java index c24d0846b5..6e5a50de9f 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/ImportTransform.java @@ -25,6 +25,7 @@ import com.google.cloud.teleport.spanner.ddl.Ddl; import com.google.cloud.teleport.spanner.ddl.NamedSchema; import com.google.cloud.teleport.spanner.ddl.Placement; +import com.google.cloud.teleport.spanner.ddl.PropertyGraph; import com.google.cloud.teleport.spanner.ddl.Sequence; import com.google.cloud.teleport.spanner.ddl.Table; import com.google.cloud.teleport.spanner.proto.ExportProtos.Export; @@ -490,6 +491,7 @@ public void processElement(ProcessContext c) { List> missingChangeStreams = new ArrayList<>(); List> missingSequences = new ArrayList<>(); List> missingPlacements = new ArrayList<>(); + List> missingPropertyGraphs = new ArrayList<>(); for (KV kv : avroSchemas) { if (informationSchemaDdl.schema(kv.getKey()) == null && informationSchemaDdl.table(kv.getKey()) == null @@ -497,7 +499,8 @@ public void processElement(ProcessContext c) { && informationSchemaDdl.view(kv.getKey()) == null && informationSchemaDdl.changeStream(kv.getKey()) == null && informationSchemaDdl.sequence(kv.getKey()) == null - && informationSchemaDdl.placement(kv.getKey()) == null) { + && informationSchemaDdl.placement(kv.getKey()) == null + && informationSchemaDdl.propertyGraph(kv.getKey()) == null) { Schema schema = parser.parse(kv.getValue()); if (schema.getProp(AvroUtil.SPANNER_CHANGE_STREAM_FOR_CLAUSE) != null) { @@ -514,6 +517,9 @@ public void processElement(ProcessContext c) { missingNamedSchemas.add(KV.of(kv.getKey(), schema)); } else if ("Placement".equals(schema.getProp("spannerEntity"))) { missingPlacements.add(KV.of(kv.getKey(), schema)); + } else if (AvroUtil.SPANNER_ENTITY_PROPERTY_GRAPH.equals( + schema.getProp("spannerEntity"))) { + missingPropertyGraphs.add(KV.of(kv.getKey(), schema)); } else { missingTables.add(KV.of(kv.getKey(), schema)); } @@ -591,7 +597,8 @@ public void processElement(ProcessContext c) { if (!missingTables.isEmpty() || !missingModels.isEmpty() - || !missingViews.isEmpty()) { + || !missingViews.isEmpty() + || !missingPropertyGraphs.isEmpty()) { Ddl.Builder builder = Ddl.builder(dialect); for (KV kv : missingViews) { com.google.cloud.teleport.spanner.ddl.View view = @@ -613,10 +620,17 @@ public void processElement(ProcessContext c) { createIndexStatements.addAll(table.indexes()); createForeignKeyStatements.addAll(table.foreignKeys()); } + for (KV kv : missingPropertyGraphs) { + PropertyGraph graph = + converter.toPropertyGraph(kv.getKey(), kv.getValue()); + builder.addPropertyGraph(graph); + mergedDdl.addPropertyGraph(graph); + } Ddl newDdl = builder.build(); ddlStatements.addAll(newDdl.createTableStatements()); ddlStatements.addAll(newDdl.createModelStatements()); ddlStatements.addAll(newDdl.createViewStatements()); + ddlStatements.addAll(newDdl.createPropertyGraphStatements()); // If the total DDL statements exceed the threshold, execute the create // index statements when tables are created. // Note that foreign keys can only be created after data load @@ -656,7 +670,7 @@ public void processElement(ProcessContext c) { c.output(pendingChangeStreamsTag, createChangeStreamStatements); LOG.info( - "Applying DDL statements for schemas, tables, models and views: {}", + "Applying DDL statements for schemas, tables, models, views and property graphs: {}", ddlStatements); if (!ddlStatements.isEmpty()) { DatabaseAdminClient databaseAdminClient = @@ -684,8 +698,10 @@ public void processElement(ProcessContext c) { } else { c.output(informationSchemaDdl); } - // In case of no tables or models, add empty list - if (missingTables.isEmpty() && missingModels.isEmpty()) { + // In case of no tables, models or property graphs, add empty list + if (missingTables.isEmpty() + && missingModels.isEmpty() + && missingPropertyGraphs.isEmpty()) { c.output(pendingIndexesTag, createIndexStatements); c.output(pendingForeignKeysTag, createForeignKeyStatements); } diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java index 7ca52426b6..fcd01a8b78 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/Ddl.java @@ -50,6 +50,7 @@ public class Ddl implements Serializable { private ImmutableSortedMap tables; private ImmutableSortedMap models; + private ImmutableSortedMap propertyGraphs; private ImmutableSortedMap views; private ImmutableSortedMap changeStreams; private ImmutableSortedMap sequences; @@ -66,6 +67,7 @@ public class Ddl implements Serializable { private Ddl( ImmutableSortedMap tables, ImmutableSortedMap models, + ImmutableSortedMap propertyGraphs, ImmutableSortedMap views, ImmutableSortedMap changeStreams, ImmutableSortedMap sequences, @@ -79,6 +81,7 @@ private Ddl( Dialect dialect) { this.tables = tables; this.models = models; + this.propertyGraphs = propertyGraphs; this.views = views; this.changeStreams = changeStreams; this.sequences = sequences; @@ -154,6 +157,14 @@ public Model model(String modelName) { return models.get(modelName.toLowerCase()); } + public Collection propertyGraphs() { + return propertyGraphs.values(); + } + + public PropertyGraph propertyGraph(String propertyGraphName) { + return propertyGraphs.get(propertyGraphName.toLowerCase()); + } + public Collection views() { return views.values(); } @@ -255,6 +266,11 @@ public void prettyPrint(Appendable appendable) throws IOException { model.prettyPrint(appendable); } + for (PropertyGraph graph : propertyGraphs()) { + appendable.append("\n"); + graph.prettyPrint(appendable); + } + for (View view : views()) { appendable.append("\n"); view.prettyPrint(appendable); @@ -285,6 +301,7 @@ public List statements() { .addAll(createIndexStatements()) .addAll(addForeignKeyStatements()) .addAll(createModelStatements()) + .addAll(createPropertyGraphStatements()) .addAll(createViewStatements()) .addAll(createChangeStreamStatements()) .addAll(createPlacementStatements()) @@ -354,6 +371,14 @@ public List createModelStatements() { return result; } + public List createPropertyGraphStatements() { + List result = new ArrayList<>(propertyGraphs.size()); + for (PropertyGraph propertyGraph : propertyGraphs.values()) { + result.add(propertyGraph.prettyPrint()); + } + return result; + } + public List createViewStatements() { List result = new ArrayList<>(views.size()); for (View view : views.values()) { @@ -481,6 +506,7 @@ public static class Builder { private Map tables = Maps.newLinkedHashMap(); private Map models = Maps.newLinkedHashMap(); + private Map propertyGraphs = Maps.newLinkedHashMap(); private Map views = Maps.newLinkedHashMap(); private Map changeStreams = Maps.newLinkedHashMap(); private Map sequences = Maps.newLinkedHashMap(); @@ -537,6 +563,26 @@ public boolean hasModel(String name) { return models.containsKey(name.toLowerCase()); } + public PropertyGraph.Builder createPropertyGraph(String name) { + PropertyGraph graph = propertyGraphs.get(name.toLowerCase()); + if (graph == null) { + return PropertyGraph.builder(dialect).name(name).ddlBuilder(this); + } + return graph.toBuilder().ddlBuilder(this); + } + + public void addPropertyGraph(PropertyGraph graph) { + propertyGraphs.put(graph.name().toLowerCase(), graph); + } + + public boolean hasPropertyGraph(String name) { + return propertyGraphs.containsKey(name.toLowerCase()); + } + + public Collection propertyGraphs() { + return propertyGraphs.values(); + } + public View.Builder createView(String name) { View view = views.get(name.toLowerCase()); if (view == null) { @@ -652,6 +698,7 @@ public Ddl build() { return new Ddl( ImmutableSortedMap.copyOf(tables), ImmutableSortedMap.copyOf(models), + ImmutableSortedMap.copyOf(propertyGraphs), ImmutableSortedMap.copyOf(views), ImmutableSortedMap.copyOf(changeStreams), ImmutableSortedMap.copyOf(sequences), @@ -671,6 +718,7 @@ public Builder toBuilder() { builder.schemas.putAll(schemas); builder.tables.putAll(tables); builder.models.putAll(models); + builder.propertyGraphs.putAll(propertyGraphs); builder.views.putAll(views); builder.changeStreams.putAll(changeStreams); builder.sequences.putAll(sequences); @@ -711,6 +759,11 @@ public boolean equals(Object o) { if (models != null ? !models.equals(ddl.models) : ddl.models != null) { return false; } + if (propertyGraphs != null + ? !propertyGraphs.equals(ddl.propertyGraphs) + : ddl.propertyGraphs != null) { + return false; + } if (views != null ? !views.equals(ddl.views) : ddl.views != null) { return false; } @@ -740,6 +793,7 @@ public int hashCode() { result = 31 * result + (parents != null ? parents.hashCode() : 0); result = 31 * result + (referencedTables != null ? referencedTables.hashCode() : 0); result = 31 * result + (models != null ? models.hashCode() : 0); + result = 31 * result + (propertyGraphs != null ? propertyGraphs.hashCode() : 0); result = 31 * result + (views != null ? views.hashCode() : 0); result = 31 * result + (changeStreams != null ? changeStreams.hashCode() : 0); result = 31 * result + (sequences != null ? sequences.hashCode() : 0); diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/GraphElementTable.java b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/GraphElementTable.java new file mode 100644 index 0000000000..20a2a3624d --- /dev/null +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/GraphElementTable.java @@ -0,0 +1,246 @@ +/* + * Copyright (C) 2024 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.cloud.spanner.Dialect; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.io.IOException; +import java.io.Serializable; +import java.util.LinkedHashMap; +import java.util.StringJoiner; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +@AutoValue +public abstract class GraphElementTable implements Serializable { + private static final long serialVersionUID = 1L; + + public enum Kind { + UNSPECIFIED, + NODE, + EDGE + } + + @Nullable + public abstract String name(); + + @Nullable + public abstract String baseTableName(); + + public abstract Kind kind(); + + public abstract Dialect dialect(); + + public abstract ImmutableList keyColumns(); + + public static class PropertyDefinition implements Serializable { + public PropertyDefinition(String name, String valueExpressionString) { + this.name = name; + this.valueExpressionString = valueExpressionString; + } + + public String name; + public String valueExpressionString; + } + + public static class LabelToPropertyDefinitions implements Serializable { + public LabelToPropertyDefinitions( + String labelName, ImmutableList propertyDefinitions) { + this.labelName = labelName; + this.propertyDefinitions = propertyDefinitions; + } + + public String labelName; + + public ImmutableList propertyDefinitions() { + return propertyDefinitions; + } + + public PropertyDefinition getPropertyDefinition(String givenPropertyDefinitionName) { + for (PropertyDefinition propertyDefinition : propertyDefinitions()) { + if (givenPropertyDefinitionName.equals(propertyDefinition.name)) { + return propertyDefinition; + } + } + return null; + } + + // A propertyDefinition is a and its + public ImmutableList propertyDefinitions; + + public String prettyPrint() { + StringBuilder sb = new StringBuilder(); + sb.append("LABEL ").append(labelName); + StringJoiner propertyJoiner = new StringJoiner(", ", " PROPERTIES(", ")"); + for (PropertyDefinition propertyDefinition : propertyDefinitions) { + String propertyName = propertyDefinition.name; + String valueExpressionString = propertyDefinition.valueExpressionString; + if (valueExpressionString.equals(propertyName)) { + propertyJoiner.add(propertyName); + } else { + StringBuilder aliasedProperty = new StringBuilder(); + aliasedProperty.append(valueExpressionString).append(" AS ").append(propertyName); + propertyJoiner.add(aliasedProperty); + } + } + sb.append(propertyDefinitions.isEmpty() ? " NO PROPERTIES" : propertyJoiner.toString()); + return sb.toString(); + } + } + + public abstract ImmutableList labelToPropertyDefinitions(); + + public LabelToPropertyDefinitions getLabelToPropertyDefinitions(String labelName) { + for (LabelToPropertyDefinitions labelToPropertyDefinitions : labelToPropertyDefinitions()) { + if (labelName.equals(labelToPropertyDefinitions.labelName)) { + return labelToPropertyDefinitions; + } + } + return null; + } + + public static class GraphNodeTableReference implements Serializable { + public GraphNodeTableReference( + String nodeTableName, + ImmutableList nodeKeyColumns, + ImmutableList edgeKeyColumns) { + this.nodeTableName = nodeTableName; + this.nodeKeyColumns = nodeKeyColumns; + this.edgeKeyColumns = edgeKeyColumns; + } + + public String nodeTableName; + public ImmutableList nodeKeyColumns; + public ImmutableList edgeKeyColumns; + + public String prettyPrint() { + StringBuilder sb = new StringBuilder(); + sb.append("KEY("); + sb.append(edgeKeyColumns.stream().collect(Collectors.joining(", "))); + sb.append(") REFERENCES ").append(nodeTableName); + return sb.toString(); + } + } + + public abstract GraphNodeTableReference sourceNodeTable(); + + public abstract GraphNodeTableReference targetNodeTable(); + + public static GraphElementTable.Builder builder() { + return builder(Dialect.GOOGLE_STANDARD_SQL); + } + + public static GraphElementTable.Builder builder(Dialect dialect) { + return new AutoValue_GraphElementTable.Builder() + .dialect(dialect) + .kind(Kind.UNSPECIFIED) + .keyColumns(ImmutableList.of()) + .labelToPropertyDefinitions(ImmutableList.of()) + .sourceNodeTable(new GraphNodeTableReference("", ImmutableList.of(), ImmutableList.of())) + .targetNodeTable(new GraphNodeTableReference("", ImmutableList.of(), ImmutableList.of())); + } + + public abstract GraphElementTable.Builder autoToBuilder(); + + public void prettyPrint(Appendable appendable) throws IOException { + if (dialect() != Dialect.GOOGLE_STANDARD_SQL) { + throw new IllegalArgumentException(String.format("Unrecognized Dialect: %s.", dialect())); + } + appendable.append(baseTableName()); + // Add alias if present + if (!name().isEmpty()) { + appendable.append(" AS ").append(name()).append("\n"); + } + // Key columns + String keyColumnsString = keyColumns().stream().collect(Collectors.joining(", ")); + appendable.append(" KEY (").append(keyColumnsString).append(")\n"); + // Source and target references for EDGE kind + if (kind() == Kind.EDGE) { + appendable + .append("SOURCE ") + .append(sourceNodeTable().prettyPrint()) + .append(" DESTINATION ") + .append(targetNodeTable().prettyPrint()) + .append("\n"); + } + // Labels and associated properties + appendable.append( + String.join( + "\n", + labelToPropertyDefinitions().stream() + .map(LabelToPropertyDefinitions::prettyPrint) + .collect(Collectors.toList()))); + } + + 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(); + } + + @AutoValue.Builder + public abstract static class Builder { + private PropertyGraph.Builder propertyGraphBuilder; + + Builder propertyGraphBuilder(PropertyGraph.Builder propertyGraphBuilder) { + this.propertyGraphBuilder = propertyGraphBuilder; + return this; + } + + private LinkedHashMap labelToPropertyDefinitions = + Maps.newLinkedHashMap(); + + public abstract GraphElementTable.Builder name(String name); + + public abstract GraphElementTable.Builder baseTableName(String baseTableName); + + public abstract GraphElementTable.Builder kind(Kind kind); + + public abstract GraphElementTable.Builder dialect(Dialect dialect); + + public abstract Builder keyColumns(ImmutableList keyColumns); + + public abstract Builder labelToPropertyDefinitions( + ImmutableList labelToPropertyDefinitions); + + public abstract Builder sourceNodeTable(GraphNodeTableReference sourceNodeTable); + + public abstract Builder targetNodeTable(GraphNodeTableReference targetNodeTable); + + public abstract GraphElementTable autoBuild(); + + public PropertyGraph.Builder endAddNodeTable() { + propertyGraphBuilder.addNodeTable(this.autoBuild()); + return propertyGraphBuilder; + } + + public PropertyGraph.Builder endAddEdgeTable() { + propertyGraphBuilder.addEdgeTable(this.autoBuild()); + return propertyGraphBuilder; + } + } +} diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java index 7e7083bb5b..91dd9c2e39 100644 --- a/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScanner.java @@ -41,7 +41,9 @@ import com.google.protobuf.DescriptorProtos.MessageOptions; import com.google.protobuf.InvalidProtocolBufferException; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -51,6 +53,8 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.beam.sdk.values.KV; +import org.json.JSONArray; +import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,6 +108,13 @@ public Ddl scan() { if (placementsSupported()) { listPlacements(builder); } + if (isPropertyGraphSupported()) { + listPropertyGraphs(builder); + listPropertyGraphPropertyDeclarations(builder); + listPropertyGraphLabels(builder); + listPropertyGraphNodeTables(builder); + listPropertyGraphEdgeTables(builder); + } Map> indexes = Maps.newHashMap(); listIndexes(indexes); listIndexColumns(builder, indexes); @@ -917,6 +928,271 @@ private boolean isModelSupported() { return dialect == Dialect.GOOGLE_STANDARD_SQL; } + private boolean isPropertyGraphSupported() { + return dialect == Dialect.GOOGLE_STANDARD_SQL; + } + + private void listPropertyGraphs(Ddl.Builder builder) { + ResultSet resultSet = + context.executeQuery( + Statement.of( + "SELECT t.property_graph_schema, t.property_graph_name " + + " FROM information_schema.property_graphs AS t " + + " WHERE t.property_graph_schema NOT IN ('INFORMATION_SCHEMA', 'SPANNER_SYS')")); + + while (resultSet.next()) { + String propertyGraphName = getQualifiedName(resultSet.getString(0), resultSet.getString(1)); + LOG.debug("Schema PropertyGraph {}", propertyGraphName); + builder.createPropertyGraph(propertyGraphName).endPropertyGraph(); + } + } + + private void listPropertyGraphPropertyDeclarations(Ddl.Builder builder) { + ResultSet resultSet = + context.executeQuery( + Statement.of( + "SELECT t.property_graph_schema, t.property_graph_name, " + + "t.property_graph_metadata_json.propertyDeclarations " + + "FROM information_schema.property_graphs AS t " + + "WHERE t.property_graph_schema NOT IN ('INFORMATION_SCHEMA', 'SPANNER_SYS')")); + + while (resultSet.next()) { + String propertyGraphSchema = resultSet.getString(0); + String propertyGraphName = resultSet.getString(1); + String propertyGraphNameQualified = getQualifiedName(propertyGraphSchema, propertyGraphName); + String propertyDeclarationsJson = resultSet.getJson(2); + + LOG.debug("Schema PropertyGraph {}", propertyGraphNameQualified); + + try { + JSONArray propertyDeclarationsArray = new JSONArray(propertyDeclarationsJson); + + for (int i = 0; i < propertyDeclarationsArray.length(); i++) { + JSONObject propertyDeclaration = propertyDeclarationsArray.getJSONObject(i); + + String name = propertyDeclaration.getString("name"); + String type = propertyDeclaration.getString("type"); + + builder + .createPropertyGraph(propertyGraphNameQualified) + .addPropertyDeclaration(new PropertyGraph.PropertyDeclaration(name, type)) + .endPropertyGraph(); + } + } catch (Exception e) { + LOG.error("Error parsing property declarations JSON: {}", e.getMessage()); + } + } + } + + private void listPropertyGraphLabels(Ddl.Builder builder) { + ResultSet resultSet = + context.executeQuery( + Statement.of( + "SELECT t.property_graph_schema, t.property_graph_name, " + + "t.property_graph_metadata_json.labels " + + "FROM information_schema.property_graphs AS t " + + "WHERE t.property_graph_schema NOT IN ('INFORMATION_SCHEMA', 'SPANNER_SYS')")); + + while (resultSet.next()) { + String propertyGraphSchema = resultSet.getString(0); + String propertyGraphName = resultSet.getString(1); + String propertyGraphNameQualified = getQualifiedName(propertyGraphSchema, propertyGraphName); + String labelsJson = resultSet.getJson(2); + + LOG.debug("Schema PropertyGraph {}", propertyGraphNameQualified); + + try { + JSONArray labelsArray = new JSONArray(labelsJson); + + for (int i = 0; i < labelsArray.length(); i++) { + JSONObject label = labelsArray.getJSONObject(i); + String name = label.getString("name"); + JSONArray propertyDeclarationNamesArray = label.getJSONArray("propertyDeclarationNames"); + + List propertyNames = new ArrayList<>(); + for (int j = 0; j < propertyDeclarationNamesArray.length(); j++) { + String propertyName = propertyDeclarationNamesArray.getString(j); + propertyNames.add(propertyName); + } + + ImmutableList immutablePropertyNames = ImmutableList.copyOf(propertyNames); + PropertyGraph.GraphElementLabel elementLabel = + new PropertyGraph.GraphElementLabel(name, immutablePropertyNames); + + builder + .createPropertyGraph(propertyGraphNameQualified) + .addLabel(elementLabel) + .endPropertyGraph(); + } + } catch (Exception e) { + LOG.error("Error parsing labels JSON: {}", e.getMessage()); + } + } + } + + public static PropertyGraph getPropertyGraphByName( + Collection graphs, String propertyGraphName) { + return graphs.stream() + .filter(graph -> graph.name().equals(propertyGraphName)) + .findFirst() + .orElse(null); + } + + private void listPropertyGraphTables(Ddl.Builder builder, String tableType) { + ResultSet resultSet = + context.executeQuery( + Statement.of( + "SELECT t.property_graph_schema, t.property_graph_name, " + + "t.property_graph_metadata_json." + + tableType + + " FROM information_schema.property_graphs AS t " + + "WHERE t.property_graph_schema NOT IN ('INFORMATION_SCHEMA', 'SPANNER_SYS')")); + + while (resultSet.next()) { + String propertyGraphSchema = resultSet.getString(0); + String propertyGraphName = resultSet.getString(1); + String propertyGraphNameQualified = getQualifiedName(propertyGraphSchema, propertyGraphName); + + String tablesJson; + try { + tablesJson = resultSet.getJson(2); + } catch (Exception edgeTableException) { + LOG.debug(propertyGraphNameQualified + " does not contain any edge tables"); + return; + } + + LOG.debug("Schema PropertyGraph {}", propertyGraphNameQualified); + + try { + JSONArray tablesArray = new JSONArray(tablesJson); + + PropertyGraph propertyGraph = + getPropertyGraphByName(builder.build().propertyGraphs(), propertyGraphNameQualified); + PropertyGraph.Builder propertyGraphBuilder = propertyGraph.toBuilder(); + if (propertyGraph == null) { + throw new RuntimeException("Property graph not found: " + propertyGraphNameQualified); + } + + for (int i = 0; i < tablesArray.length(); i++) { + JSONObject table = tablesArray.getJSONObject(i); + + String baseTableName = table.getString("baseTableName"); + JSONArray keyColumnsArray = table.getJSONArray("keyColumns"); + String kind = table.getString("kind"); + JSONArray labelNamesArray = table.getJSONArray("labelNames"); + String name = table.getString("name"); + JSONArray propertyDefinitionsArray = table.getJSONArray("propertyDefinitions"); + + ImmutableList.Builder keyColumnsBuilder = ImmutableList.builder(); + for (int j = 0; j < keyColumnsArray.length(); j++) { + keyColumnsBuilder.add(keyColumnsArray.getString(j)); + } + ImmutableList keyColumns = keyColumnsBuilder.build(); + + GraphElementTable.Builder graphElementTableBuilder = + GraphElementTable.builder() + .propertyGraphBuilder(propertyGraphBuilder) + .name(name) + .baseTableName(baseTableName) + .kind(GraphElementTable.Kind.valueOf(kind)) + .keyColumns(keyColumns); + + // If it's an edge table, extract source and destination node table references + if (tableType.equals("edgeTables")) { + JSONObject sourceNodeTable = table.getJSONObject("sourceNodeTable"); + JSONObject destinationNodeTable = table.getJSONObject("destinationNodeTable"); + + GraphElementTable.GraphNodeTableReference sourceNodeTableReference = + new GraphElementTable.GraphNodeTableReference( + sourceNodeTable.getString("nodeTableName"), + ImmutableList.copyOf( + toStringList(sourceNodeTable.getJSONArray("nodeTableColumns"))), + ImmutableList.copyOf( + toStringList(sourceNodeTable.getJSONArray("edgeTableColumns")))); + + GraphElementTable.GraphNodeTableReference destinationNodeTableReference = + new GraphElementTable.GraphNodeTableReference( + destinationNodeTable.getString("nodeTableName"), + ImmutableList.copyOf( + toStringList(destinationNodeTable.getJSONArray("nodeTableColumns"))), + ImmutableList.copyOf( + toStringList(destinationNodeTable.getJSONArray("edgeTableColumns")))); + + graphElementTableBuilder + .sourceNodeTable(sourceNodeTableReference) + .targetNodeTable(destinationNodeTableReference); + } + + List labelsToPropertyDefinitions = + new ArrayList<>(); + for (int j = 0; j < labelNamesArray.length(); j++) { + String labelName = labelNamesArray.getString(j); + + PropertyGraph.GraphElementLabel propertyGraphLabel = propertyGraph.getLabel(labelName); + + if (propertyGraphLabel != null) { + ImmutableList.Builder + propertyDefinitionsBuilder = ImmutableList.builder(); + + for (String propertyName : propertyGraphLabel.properties) { + for (int k = 0; k < propertyDefinitionsArray.length(); k++) { + JSONObject propertyDefinition = propertyDefinitionsArray.getJSONObject(k); + String propertyDeclarationName = + propertyDefinition.getString("propertyDeclarationName"); + + if (propertyName.equals(propertyDeclarationName)) { + PropertyGraph.PropertyDeclaration propertyDeclaration = + propertyGraph.getPropertyDeclaration(propertyDeclarationName); + propertyDefinitionsBuilder.add( + new GraphElementTable.PropertyDefinition( + propertyDeclaration.name, + propertyDefinition.getString("valueExpressionSql"))); + break; + } + } + } + ImmutableList propertyDefinitions = + propertyDefinitionsBuilder.build(); + labelsToPropertyDefinitions.add( + new GraphElementTable.LabelToPropertyDefinitions(labelName, propertyDefinitions)); + } + } + graphElementTableBuilder.labelToPropertyDefinitions( + ImmutableList.copyOf(labelsToPropertyDefinitions)); + + // Add the GraphElementTable to the PropertyGraph builder + if (tableType.equals("nodeTables")) { + propertyGraphBuilder.addNodeTable(graphElementTableBuilder.autoBuild()); + } else { // tableType.equals("edgeTables") + propertyGraphBuilder.addEdgeTable(graphElementTableBuilder.autoBuild()); + } + } + propertyGraph = propertyGraphBuilder.build(); + builder.addPropertyGraph(propertyGraph); + + } catch (Exception e) { + LOG.error("Error parsing {} JSON: {}", tableType, e.getMessage()); + } + } + } + + // Helper function to convert JSONArray to List + private static List toStringList(JSONArray jsonArray) { + List list = new ArrayList<>(); + for (int i = 0; i < jsonArray.length(); i++) { + list.add(jsonArray.getString(i)); + } + return list; + } + + private void listPropertyGraphNodeTables(Ddl.Builder builder) { + listPropertyGraphTables(builder, "nodeTables"); + } + + private void listPropertyGraphEdgeTables(Ddl.Builder builder) { + listPropertyGraphTables(builder, "edgeTables"); + } + private void listModels(Ddl.Builder builder) { ResultSet resultSet = context.executeQuery( diff --git a/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/PropertyGraph.java b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/PropertyGraph.java new file mode 100644 index 0000000000..b82c536160 --- /dev/null +++ b/v1/src/main/java/com/google/cloud/teleport/spanner/ddl/PropertyGraph.java @@ -0,0 +1,234 @@ +/* + * Copyright (C) 2024 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.cloud.spanner.Dialect; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.io.IOException; +import java.io.Serializable; +import java.util.LinkedHashMap; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +@AutoValue +public abstract class PropertyGraph implements Serializable { + private static final long serialVersionUID = 1L; + + @Nullable + public abstract String name(); + + public abstract ImmutableList nodeTables(); + + public abstract ImmutableList edgeTables(); + + public static class PropertyDeclaration implements Serializable { + public PropertyDeclaration(String name, String type) { + this.name = name; + this.type = type; + } + + public String name; + public String type; + } + + public abstract ImmutableList propertyDeclarations(); + + public PropertyDeclaration getPropertyDeclaration(String givenPropertyDeclarationName) { + for (PropertyDeclaration propertyDeclaration : propertyDeclarations()) { + if (givenPropertyDeclarationName.equals(propertyDeclaration.name)) { + return propertyDeclaration; + } + } + return null; + } + + public GraphElementLabel getLabel(String givenLabelName) { + for (GraphElementLabel label : labels()) { + if (givenLabelName.equals(label.name)) { + return label; + } + } + return null; + } + + public GraphElementTable getNodeTable(String givenNodeTableName) { + for (GraphElementTable nodeTable : nodeTables()) { + if (givenNodeTableName.equals(nodeTable.name())) { + return nodeTable; + } + } + return null; + } + + public GraphElementTable getEdgeTable(String givenEdgeTableName) { + for (GraphElementTable edgeTable : edgeTables()) { + if (givenEdgeTableName.equals(edgeTable.name())) { + return edgeTable; + } + } + return null; + } + + public static class GraphElementLabel implements Serializable { + public GraphElementLabel(String name, ImmutableList properties) { + this.name = name; + this.properties = properties; + } + + public String name; + public ImmutableList properties; + } + + public abstract ImmutableList labels(); + + public abstract Dialect dialect(); + + public static PropertyGraph.Builder builder() { + return builder(Dialect.GOOGLE_STANDARD_SQL); + } + + public static PropertyGraph.Builder builder(Dialect dialect) { + return new AutoValue_PropertyGraph.Builder().dialect(dialect); + } + + public abstract PropertyGraph.Builder autoToBuilder(); + + public PropertyGraph.Builder toBuilder() { + PropertyGraph.Builder builder = autoToBuilder(); + builder = builder.dialect(dialect()); + + for (GraphElementLabel label : labels()) { + builder.addLabel(label); + } + for (PropertyDeclaration declaration : propertyDeclarations()) { + builder.addPropertyDeclaration(declaration); + } + for (GraphElementTable nodeTable : nodeTables()) { + builder.addNodeTable(nodeTable); + } + for (GraphElementTable edgeTable : edgeTables()) { + builder.addEdgeTable(edgeTable); + } + return builder; + } + + @Override + public String toString() { + return prettyPrint(); + } + + public void prettyPrint(Appendable appendable) throws IOException { + appendable.append("CREATE PROPERTY GRAPH ").append(name()); + appendable.append("\nNODE TABLES(\n"); + appendable.append( + String.join( + ", ", + nodeTables().stream() + .map(GraphElementTable::prettyPrint) + .collect(Collectors.toList()))); + appendable.append(")"); // End NODE TABLES() + if (edgeTables().size() > 0) { + appendable.append("\nEDGE TABLES(\n"); + appendable.append( + String.join( + ", ", + edgeTables().stream() + .map(GraphElementTable::prettyPrint) + .collect(Collectors.toList()))); + appendable.append(")"); // End EDGE TABLES() + } + } + + public String prettyPrint() { + StringBuilder sb = new StringBuilder(); + try { + prettyPrint(sb); + } catch (IOException e) { + throw new RuntimeException(e); + } + return sb.toString(); + } + + @AutoValue.Builder + public abstract static class Builder { + private Ddl.Builder ddlBuilder; + + private LinkedHashMap nodeTables = Maps.newLinkedHashMap(); + private LinkedHashMap edgeTables = Maps.newLinkedHashMap(); + private LinkedHashMap propertyDeclarations = + Maps.newLinkedHashMap(); + private LinkedHashMap labels = Maps.newLinkedHashMap(); + + public PropertyGraph.Builder ddlBuilder(Ddl.Builder ddlBuilder) { + this.ddlBuilder = ddlBuilder; + return this; + } + + abstract PropertyGraph.Builder nodeTables(ImmutableList nodeTables); + + abstract PropertyGraph.Builder edgeTables(ImmutableList edgeTables); + + abstract PropertyGraph.Builder propertyDeclarations( + ImmutableList propertyDeclarations); + + abstract PropertyGraph.Builder labels(ImmutableList labels); + + public abstract PropertyGraph.Builder name(String name); + + public abstract String name(); + + public abstract PropertyGraph.Builder dialect(Dialect dialect); + + public abstract Dialect dialect(); + + abstract PropertyGraph autoBuild(); + + public PropertyGraph build() { + return nodeTables(ImmutableList.copyOf(nodeTables.values())) + .edgeTables(ImmutableList.copyOf(edgeTables.values())) + .propertyDeclarations(ImmutableList.copyOf(propertyDeclarations.values())) + .labels(ImmutableList.copyOf(labels.values())) + .autoBuild(); + } + + public PropertyGraph.Builder addNodeTable(GraphElementTable elementTable) { + nodeTables.put(elementTable.name().toLowerCase(), elementTable); + return this; + } + + public PropertyGraph.Builder addEdgeTable(GraphElementTable elementTable) { + edgeTables.put(elementTable.name().toLowerCase(), elementTable); + return this; + } + + public PropertyGraph.Builder addPropertyDeclaration(PropertyDeclaration propertyDeclaration) { + propertyDeclarations.put(propertyDeclaration.name.toLowerCase(), propertyDeclaration); + return this; + } + + public PropertyGraph.Builder addLabel(GraphElementLabel label) { + labels.put(label.name.toLowerCase(), label); + return this; + } + + public Ddl.Builder endPropertyGraph() { + ddlBuilder.addPropertyGraph(build()); + return ddlBuilder; + } + } +} diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java index 262b7cc560..4be7a46a48 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/AvroSchemaToDdlConverterTest.java @@ -453,6 +453,72 @@ public void pgSimple() { + " REFERENCES \"AllowedNames\" (\"last_name\") ON DELETE CASCADE")); } + @Test + public void propertyGraphs() { + String avroString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"testGraph\",\n" + + " \"namespace\": \"spannertest\",\n" + + " \"fields\": [],\n" + + " \"spannerGraphNodeTable_0_NAME\": \"nodeAlias\",\n" + + " \"spannerGraphNodeTable_0_BASE_TABLE_NAME\": \"baseTable\",\n" + + " \"spannerName\": \"testGraph\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_NAME\": \"dummyLabelName1\",\n" + + " \"spannerEntity\": \"PropertyGraph\",\n" + + " \"spannerGraphLabel_1_NAME\": \"dummyLabelName2\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_PROPERTY_0_NAME\": \"dummyPropName\",\n" + + " \"spannerGraphEdgeTable_0_KEY_COLUMNS\": \"edgePrimaryKey\",\n" + + " \"spannerGraphEdgeTable_0_TARGET_NODE_KEY_COLUMNS\": \"otherNodeKey\",\n" + + " \"googleStorage\": \"CloudSpanner\",\n" + + " \"spannerGraphEdgeTable_0_SOURCE_NODE_KEY_COLUMNS\": \"nodeKey\",\n" + + " \"spannerGraphNodeTable_0_KIND\": \"NODE\",\n" + + " \"spannerGraphNodeTable_0_LABEL_1_NAME\": \"dummyLabelName2\",\n" + + " \"spannerGraphEdgeTable_0_NAME\": \"edgeAlias\",\n" + + " \"spannerGraphEdgeTable_0_BASE_TABLE_NAME\": \"edgeBaseTable\",\n" + + " \"spannerGraphEdgeTable_0_KIND\": \"EDGE\",\n" + + " \"spannerGraphLabel_0_PROPERTY_1\": \"aliasedPropName\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_PROPERTY_1_NAME\": \"aliasedPropName\",\n" + + " \"spannerGraphEdgeTable_0_LABEL_0_NAME\": \"dummyLabelName3\",\n" + + " \"spannerGraphNodeTable_0_KEY_COLUMNS\": \"primaryKey\",\n" + + " \"spannerGraphLabel_2_NAME\": \"dummyLabelName3\",\n" + + " \"spannerGraphLabel_0_PROPERTY_0\": \"dummyPropName\",\n" + + " \"spannerGraphEdgeTable_0_SOURCE_EDGE_KEY_COLUMNS\": \"sourceEdgeKey\",\n" + + " \"spannerGraphEdgeTable_0_TARGET_EDGE_KEY_COLUMNS\": \"destEdgeKey\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_PROPERTY_0_VALUE\": \"dummyPropName\",\n" + + " \"spannerGraphEdgeTable_0_TARGET_NODE_TABLE_NAME\": \"baseTable\",\n" + + " \"spannerGraphEdgeTable_0_SOURCE_NODE_TABLE_NAME\": \"baseTable\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_PROPERTY_1_VALUE\": \"CONCAT(CAST(test_col AS STRING), \\\":\\\", \\\"dummyColumn\\\")\",\n" + + " \"spannerGraphLabel_0_NAME\": \"dummyLabelName1\",\n" + + " \"googleFormatVersion\": \"booleans\",\n" + + " \"spannerGraphPropertyDeclaration_1_NAME\": \"aliasedPropName\",\n" + + " \"spannerGraphPropertyDeclaration_1_TYPE\": \"dummyPropType\",\n" + + " \"spannerGraphPropertyDeclaration_0_NAME\": \"aliasedPropName\",\n" + + " \"spannerGraphPropertyDeclaration_0_TYPE\": \"dummyPropType\"\n" + + "}"; + + Schema schema = new Schema.Parser().parse(avroString); + + AvroSchemaToDdlConverter converter = new AvroSchemaToDdlConverter(); + Ddl ddl = converter.toDdl(Collections.singleton(schema)); + assertThat(ddl.propertyGraphs(), hasSize(1)); + + String expectedPg = + "CREATE PROPERTY GRAPH testGraph\n" + + "NODE TABLES(\n" + + "baseTable AS nodeAlias\n" + + " KEY (primaryKey)\n" + + "LABEL dummyLabelName1 PROPERTIES(dummyPropName, CONCAT(CAST(test_col AS STRING), \":\", \"dummyColumn\") AS aliasedPropName)\n" + + "LABEL dummyLabelName2 NO PROPERTIES)\n" + + "EDGE TABLES(\n" + + "edgeBaseTable AS edgeAlias\n" + + " KEY (edgePrimaryKey)\n" + + "SOURCE KEY(sourceEdgeKey) REFERENCES baseTable DESTINATION KEY(destEdgeKey) REFERENCES baseTable\n" + + "LABEL dummyLabelName3 NO PROPERTIES)"; + + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(expectedPg)); + } + @Test public void models() { String modelAllString = diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java index fe29aa6df2..acefcf177a 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/DdlToAvroSchemaConverterTest.java @@ -25,16 +25,21 @@ import static com.google.cloud.teleport.spanner.AvroUtil.OUTPUT; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_CHANGE_STREAM_FOR_CLAUSE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_CHECK_CONSTRAINT; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_EDGE_TABLE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_MODEL; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_PLACEMENT; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ENTITY_PROPERTY_GRAPH; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_FOREIGN_KEY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_INDEX; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_LABEL; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_NODE_TABLE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_ON_DELETE_ACTION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_OPTION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PARENT; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PLACEMENT_KEY; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PRIMARY_KEY; +import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_PROPERTY_DECLARATION; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_REMOTE; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_SEQUENCE_COUNTER_START; import static com.google.cloud.teleport.spanner.AvroUtil.SPANNER_SEQUENCE_KIND; @@ -49,6 +54,7 @@ import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import com.google.cloud.spanner.Dialect; @@ -56,8 +62,14 @@ import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.common.Type.StructField; import com.google.cloud.teleport.spanner.ddl.Ddl; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.GraphNodeTableReference; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.LabelToPropertyDefinitions; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.PropertyDefinition; +import com.google.cloud.teleport.spanner.ddl.PropertyGraph; import com.google.cloud.teleport.spanner.ddl.View; import com.google.common.collect.ImmutableList; +import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -1059,6 +1071,153 @@ public void pgTimestampLogicalTypeTest() { assertThat(fields.get(1).getProp(SQL_TYPE), equalTo("timestamp with time zone")); } + @Test + public void propertyGraphs() { + DdlToAvroSchemaConverter converter = + new DdlToAvroSchemaConverter("spannertest", "booleans", true); + + // Craft Property Declarations + PropertyGraph.PropertyDeclaration propertyDeclaration1 = + new PropertyGraph.PropertyDeclaration("dummyPropName", "dummyPropType"); + PropertyGraph.PropertyDeclaration propertyDeclaration2 = + new PropertyGraph.PropertyDeclaration("aliasedPropName", "aliasedPropType"); + ImmutableList propertyDeclsLabel1 = + ImmutableList.copyOf(Arrays.asList(propertyDeclaration1.name, propertyDeclaration2.name)); + + // Craft Labels and associated property definitions + PropertyGraph.GraphElementLabel label1 = + new PropertyGraph.GraphElementLabel("dummyLabelName1", propertyDeclsLabel1); + GraphElementTable.PropertyDefinition propertyDefinition1 = + new PropertyDefinition("dummyPropName", "dummyPropName"); + GraphElementTable.PropertyDefinition propertyDefinition2 = + new PropertyDefinition( + "aliasedPropName", "CONCAT(CAST(test_col AS STRING), \":\", \"dummyColumn\")"); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions1 = + new LabelToPropertyDefinitions( + label1.name, ImmutableList.of(propertyDefinition1, propertyDefinition2)); + + PropertyGraph.GraphElementLabel label2 = + new PropertyGraph.GraphElementLabel("dummyLabelName2", ImmutableList.of()); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions2 = + new LabelToPropertyDefinitions(label2.name, ImmutableList.of()); + + PropertyGraph.GraphElementLabel label3 = + new PropertyGraph.GraphElementLabel("dummyLabelName3", ImmutableList.of()); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions3 = + new LabelToPropertyDefinitions(label3.name, ImmutableList.of()); + + // Craft Node table + GraphElementTable.Builder testNodeTable = + GraphElementTable.builder() + .baseTableName("baseTable") + .name("nodeAlias") + .kind(GraphElementTable.Kind.NODE) + .keyColumns(ImmutableList.of("primaryKey")) + .labelToPropertyDefinitions( + ImmutableList.of(labelToPropertyDefinitions1, labelToPropertyDefinitions2)); + + // Craft Edge table + GraphElementTable.Builder testEdgeTable = + GraphElementTable.builder() + .baseTableName("edgeBaseTable") + .name("edgeAlias") + .kind(GraphElementTable.Kind.EDGE) + .keyColumns(ImmutableList.of("edgePrimaryKey")) + .sourceNodeTable( + new GraphNodeTableReference( + "baseTable", ImmutableList.of("nodeKey"), ImmutableList.of("sourceEdgeKey"))) + .targetNodeTable( + new GraphNodeTableReference( + "baseTable", ImmutableList.of("otherNodeKey"), ImmutableList.of("destEdgeKey"))) + .labelToPropertyDefinitions(ImmutableList.of(labelToPropertyDefinitions3)); + + Ddl ddl = + Ddl.builder() + .createPropertyGraph("testGraph") + .addLabel(label1) + .addLabel(label2) + .addLabel(label3) + .addPropertyDeclaration(propertyDeclaration1) + .addPropertyDeclaration(propertyDeclaration2) + .addNodeTable(testNodeTable.autoBuild()) + .addEdgeTable(testEdgeTable.autoBuild()) + .endPropertyGraph() + .build(); + + Collection result = converter.convert(ddl); + assertThat(result, hasSize(1)); + Schema avroSchema = result.iterator().next(); + + assertThat(avroSchema.getName(), equalTo("testGraph")); + assertThat(avroSchema.getNamespace(), equalTo("spannertest")); + assertThat(avroSchema.getProp(GOOGLE_FORMAT_VERSION), equalTo("booleans")); + assertThat(avroSchema.getProp(GOOGLE_STORAGE), equalTo("CloudSpanner")); + assertThat(avroSchema.getProp(SPANNER_ENTITY), equalTo(SPANNER_ENTITY_PROPERTY_GRAPH)); + + // Basic assertions + assertEquals("testGraph", avroSchema.getName()); + assertEquals("spannertest", avroSchema.getNamespace()); + assertEquals(Schema.Type.RECORD, avroSchema.getType()); + + // Asserting properties related to Spanner + assertEquals("CloudSpanner", avroSchema.getProp(GOOGLE_STORAGE)); + assertEquals("testGraph", avroSchema.getProp("spannerName")); + assertEquals("booleans", avroSchema.getProp(GOOGLE_FORMAT_VERSION)); + + // Asserting properties related to Node table + assertEquals("nodeAlias", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_NAME")); + assertEquals("baseTable", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_BASE_TABLE_NAME")); + assertEquals("NODE", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_KIND")); + assertEquals("primaryKey", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_KEY_COLUMNS")); + + // Asserting properties related to Node labels + assertEquals("dummyLabelName1", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_LABEL_0_NAME")); + assertEquals("dummyLabelName2", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_LABEL_1_NAME")); + + // Asserting properties related to Node label properties + assertEquals( + "dummyPropName", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_LABEL_0_PROPERTY_0_NAME")); + assertEquals( + "dummyPropName", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_LABEL_0_PROPERTY_0_VALUE")); + assertEquals( + "aliasedPropName", avroSchema.getProp(SPANNER_NODE_TABLE + "_0_LABEL_0_PROPERTY_1_NAME")); + assertEquals( + "CONCAT(CAST(test_col AS STRING), \":\", \"dummyColumn\")", + avroSchema.getProp(SPANNER_NODE_TABLE + "_0_LABEL_0_PROPERTY_1_VALUE")); + + // Asserting properties related to Edge table + assertEquals("edgeAlias", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_NAME")); + assertEquals("edgeBaseTable", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_BASE_TABLE_NAME")); + assertEquals("EDGE", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_KIND")); + assertEquals("edgePrimaryKey", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_KEY_COLUMNS")); + assertEquals("baseTable", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_SOURCE_NODE_TABLE_NAME")); + assertEquals("nodeKey", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_SOURCE_NODE_KEY_COLUMNS")); + assertEquals( + "sourceEdgeKey", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_SOURCE_EDGE_KEY_COLUMNS")); + assertEquals("baseTable", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_TARGET_NODE_TABLE_NAME")); + assertEquals( + "otherNodeKey", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_TARGET_NODE_KEY_COLUMNS")); + assertEquals( + "destEdgeKey", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_TARGET_EDGE_KEY_COLUMNS")); + + // Asserting properties related to Edge labels + assertEquals("dummyLabelName3", avroSchema.getProp(SPANNER_EDGE_TABLE + "_0_LABEL_0_NAME")); + + // Asserting labels and properties linked to them + assertEquals("dummyLabelName1", avroSchema.getProp(SPANNER_LABEL + "_0_NAME")); + assertEquals("dummyPropName", avroSchema.getProp(SPANNER_LABEL + "_0_PROPERTY_0")); + assertEquals("aliasedPropName", avroSchema.getProp(SPANNER_LABEL + "_0_PROPERTY_1")); + + assertEquals("dummyLabelName2", avroSchema.getProp(SPANNER_LABEL + "_1_NAME")); + assertEquals("dummyLabelName3", avroSchema.getProp(SPANNER_LABEL + "_2_NAME")); + + // Asserting properties related to graph property declarations + assertEquals("dummyPropName", avroSchema.getProp(SPANNER_PROPERTY_DECLARATION + "_0_NAME")); + assertEquals("dummyPropType", avroSchema.getProp(SPANNER_PROPERTY_DECLARATION + "_0_TYPE")); + assertEquals("aliasedPropName", avroSchema.getProp(SPANNER_PROPERTY_DECLARATION + "_1_NAME")); + assertEquals("aliasedPropType", avroSchema.getProp(SPANNER_PROPERTY_DECLARATION + "_1_TYPE")); + } + @Test public void models() { DdlToAvroSchemaConverter converter = diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java index 398c42b93a..4e3c00a7d2 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ExportTransformTest.java @@ -147,7 +147,9 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { "sequence", "sequence manifest", "placement", - "placement manifest"); + "placement manifest", + "propertyGraph1", + "propertyGraph1 manifest"); FileDescriptorProto.Builder builder = FileDescriptorProto.newBuilder(); builder @@ -179,6 +181,7 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { ddlBuilder.createModel("model1").remote(true).endModel(); ddlBuilder.createChangeStream("changeStream").endChangeStream(); ddlBuilder.createSequence("sequence").endSequence(); + ddlBuilder.createPropertyGraph("propertyGraph1").endPropertyGraph(); ddlBuilder .createPlacement("placement") .options( @@ -218,9 +221,12 @@ public void buildDatabaseManifestFile() throws InvalidProtocolBufferException { assertEquals(protoDescriptorsResult, manifestProto.getProtoDescriptors()); assertEquals(protoBundle, new HashSet<>(manifestProto.getProtoBundleList())); - assertThat(manifestProto.getTablesCount(), is(3)); + assertThat(manifestProto.getTablesCount(), is(4)); for (Table table : manifestProto.getTablesList()) { - assertThat(table.getName(), anyOf(startsWith("table"), startsWith("model"))); + assertThat( + table.getName(), + anyOf( + startsWith("table"), startsWith("model"), startsWith("propertyGraph"))); assertThat(table.getManifestFile(), is(table.getName() + "-manifest.json")); } diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java index fec365ff14..1064dd846c 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ImportFromAvroTest.java @@ -1298,6 +1298,92 @@ public void models() throws Exception { + " REMOTE OPTIONS (endpoint=\"//aiplatform.googleapis.com/projects/span-cloud-testing/locations/us-central1/publishers/google/models/textembedding-gecko\")")); } + @Test + public void propertyGraphs() throws Exception { + String avroString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"aml\",\n" + + " \"namespace\": \"spannerexport\",\n" + + " \"fields\": [],\n" + + " \"spannerGraphNodeTable_0_NAME\": \"Test\",\n" + + " \"spannerGraphNodeTable_0_BASE_TABLE_NAME\": \"Test\",\n" + + " \"spannerName\": \"aml\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_NAME\": \"Test\",\n" + + " \"spannerEntity\": \"PropertyGraph\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_PROPERTY_0_NAME\": \"Id\",\n" + + " \"spannerGraphNodeTable_0_LABEL_0_PROPERTY_0_VALUE\": \"Id\",\n" + + " \"googleStorage\": \"CloudSpanner\",\n" + + " \"spannerGraphNodeTable_0_KIND\": \"NODE\",\n" + + " \"spannerGraphLabel_0_NAME\": \"Test\",\n" + + " \"googleFormatVersion\": \"1.0.0\",\n" + + " \"spannerGraphNodeTable_0_KEY_COLUMNS\": \"Id\",\n" + + " \"spannerGraphPropertyDeclaration_0_NAME\": \"Id\",\n" + + " \"spannerGraphPropertyDeclaration_0_TYPE\": \"INT64\",\n" + + " \"spannerGraphLabel_0_PROPERTY_0\": \"Id\"\n" + + "}"; + + String fileName = "PropertyGraphAll.avro"; + Schema schema = new Schema.Parser().parse(avroString); + ExportProtos.Export.Builder exportProtoBuilder = ExportProtos.Export.newBuilder(); + exportProtoBuilder.addTables( + 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 Test (\n" + " Id INT64 NOT NULL,\n" + ") PRIMARY KEY(Id)"; + SPANNER_SERVER.createDatabase(dbName, Collections.singleton(spannerSchema)); + + // Run the import pipeline. + importPipeline.apply( + "Import", + new ImportTransform( + SPANNER_SERVER.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(true), + ValueProvider.StaticValueProvider.of(30))); + + PipelineResult importResult = importPipeline.run(); + importResult.waitUntilFinish(); + + Ddl ddl; + try (ReadOnlyTransaction ctx = SPANNER_SERVER.getDbClient(dbName).readOnlyTransaction()) { + ddl = new InformationSchemaScanner(ctx).scan(); + } + + String expectedPropertyGraph = + "CREATE TABLE `Test` (\n" + + "\t`Id` INT64 NOT NULL,\n" + + ") PRIMARY KEY (`Id` ASC)\n" + + "CREATE PROPERTY GRAPH aml\n" + + "NODE TABLES(\n" + + "Test AS Test\n" + + " KEY (Id)\n" + + "LABEL Test PROPERTIES(Id))"; + assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(expectedPropertyGraph)); + } + @Test public void changeStreams() throws Exception { Map avroFiles = new HashMap<>(); diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java index 20f05b6607..78de2f8b2d 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/DdlTest.java @@ -31,6 +31,9 @@ import com.google.cloud.spanner.Dialect; import com.google.cloud.teleport.spanner.common.Type; import com.google.cloud.teleport.spanner.ddl.ForeignKey.ReferentialAction; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.GraphNodeTableReference; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.LabelToPropertyDefinitions; +import com.google.cloud.teleport.spanner.ddl.GraphElementTable.PropertyDefinition; import com.google.cloud.teleport.spanner.ddl.IndexColumn.IndexColumnsBuilder; import com.google.cloud.teleport.spanner.ddl.IndexColumn.Order; import com.google.cloud.teleport.spanner.proto.ExportProtos.Export; @@ -39,6 +42,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import org.junit.Test; @@ -818,6 +822,97 @@ public void pgTestModel() { assertThrows(IllegalArgumentException.class, () -> model.build().prettyPrint()); } + @Test + public void testPropertyGraph() { + // Craft Property Declarations + PropertyGraph.PropertyDeclaration propertyDeclaration1 = + new PropertyGraph.PropertyDeclaration("dummy-prop-name", "dummy-prop-type"); + PropertyGraph.PropertyDeclaration propertyDeclaration2 = + new PropertyGraph.PropertyDeclaration("aliased-prop-name", "dummy-prop-type"); + ImmutableList propertyDeclsLabel1 = + ImmutableList.copyOf(Arrays.asList(propertyDeclaration1.name, propertyDeclaration2.name)); + + // Craft Labels and associated property definitions + PropertyGraph.GraphElementLabel label1 = + new PropertyGraph.GraphElementLabel("dummy-label-name1", propertyDeclsLabel1); + GraphElementTable.PropertyDefinition propertyDefinition1 = + new PropertyDefinition("dummy-prop-name", "dummy-prop-name"); + GraphElementTable.PropertyDefinition propertyDefinition2 = + new PropertyDefinition( + "aliased-prop-name", "CONCAT(CAST(test_col AS STRING), \":\", \"dummy-column\")"); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions1 = + new LabelToPropertyDefinitions( + label1.name, ImmutableList.of(propertyDefinition1, propertyDefinition2)); + + PropertyGraph.GraphElementLabel label2 = + new PropertyGraph.GraphElementLabel("dummy-label-name2", ImmutableList.of()); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions2 = + new LabelToPropertyDefinitions(label2.name, ImmutableList.of()); + + PropertyGraph.GraphElementLabel label3 = + new PropertyGraph.GraphElementLabel("dummy-label-name3", ImmutableList.of()); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions3 = + new LabelToPropertyDefinitions(label3.name, ImmutableList.of()); + + // Craft Node table + GraphElementTable.Builder testNodeTable = + GraphElementTable.builder() + .baseTableName("base-table") + .name("node-alias") + .kind(GraphElementTable.Kind.NODE) + .keyColumns(ImmutableList.of("primary-key")) + .labelToPropertyDefinitions( + ImmutableList.of(labelToPropertyDefinitions1, labelToPropertyDefinitions2)); + + // Craft Edge table + GraphElementTable.Builder testEdgeTable = + GraphElementTable.builder() + .baseTableName("edge-base-table") + .name("edge-alias") + .kind(GraphElementTable.Kind.EDGE) + .keyColumns(ImmutableList.of("edge-primary-key")) + .sourceNodeTable( + new GraphNodeTableReference( + "base-table", + ImmutableList.of("node-key"), + ImmutableList.of("source-edge-key"))) + .targetNodeTable( + new GraphNodeTableReference( + "base-table", + ImmutableList.of("other-node-key"), + ImmutableList.of("dest-edge-key"))) + .labelToPropertyDefinitions(ImmutableList.of(labelToPropertyDefinitions3)); + + // Build PropertyGraph + PropertyGraph.Builder propertyGraph = + PropertyGraph.builder() + .name("test-graph") + .addLabel(label1) + .addLabel(label2) + .addLabel(label3) + .addPropertyDeclaration(propertyDeclaration1) + .addPropertyDeclaration(propertyDeclaration2) + .addNodeTable(testNodeTable.autoBuild()) + .addEdgeTable(testEdgeTable.autoBuild()); + + assertThat( + propertyGraph.build().prettyPrint(), + equalToCompressingWhiteSpace( + "CREATE PROPERTY GRAPH test-graph " + + "NODE TABLES(\n" + + "base-table AS node-alias\n" + + " KEY (primary-key)\n" + + "LABEL dummy-label-name1 " + + "PROPERTIES(dummy-prop-name, CONCAT(CAST(test_col AS STRING), \":\", \"dummy-column\") AS aliased-prop-name)\n" + + "LABEL dummy-label-name2 NO PROPERTIES)\n" + + "EDGE TABLES(\n" + + "edge-base-table AS edge-alias\n" + + " KEY (edge-primary-key)\n" + + "SOURCE KEY(source-edge-key) REFERENCES base-table DESTINATION KEY(dest-edge-key) REFERENCES base-table\n" + + "LABEL dummy-label-name3 NO PROPERTIES" + + ")")); + } + @Test public void testView() { View view = View.builder().name("user_view").query("SELECT * FROM `User`").build(); diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/GraphElementTableTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/GraphElementTableTest.java new file mode 100644 index 0000000000..d99ae9ef2a --- /dev/null +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/GraphElementTableTest.java @@ -0,0 +1,136 @@ +/* + * Copyright (C) 2024 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.google.cloud.spanner.Dialect; +import com.google.common.collect.ImmutableList; +import org.junit.Test; + +public class GraphElementTableTest { + @Test + public void testPropertyDefinition() { + GraphElementTable.PropertyDefinition propertyDefinition = + new GraphElementTable.PropertyDefinition("propertyA", "valueA"); + assertEquals("propertyA", propertyDefinition.name); + assertEquals("valueA", propertyDefinition.valueExpressionString); + } + + @Test + public void testLabelToPropertyDefinitions() { + ImmutableList propertyDefinitions = + ImmutableList.of( + new GraphElementTable.PropertyDefinition("propertyA", "valueA"), + new GraphElementTable.PropertyDefinition("propertyB", "valueB")); + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions = + new GraphElementTable.LabelToPropertyDefinitions("label1", propertyDefinitions); + + assertEquals("label1", labelToPropertyDefinitions.labelName); + assertEquals(propertyDefinitions, labelToPropertyDefinitions.propertyDefinitions()); + + assertEquals( + propertyDefinitions.get(0), labelToPropertyDefinitions.getPropertyDefinition("propertyA")); + assertNull(labelToPropertyDefinitions.getPropertyDefinition("propertyC")); + + String expectedPrettyPrint = + "LABEL label1 PROPERTIES(valueA AS propertyA, valueB AS propertyB)"; + assertEquals(expectedPrettyPrint, labelToPropertyDefinitions.prettyPrint()); + + // Test prettyPrint with aliased properties + propertyDefinitions = + ImmutableList.of( + new GraphElementTable.PropertyDefinition("propertyA", "propertyA"), + new GraphElementTable.PropertyDefinition("propertyB", "valueB")); + labelToPropertyDefinitions = + new GraphElementTable.LabelToPropertyDefinitions("label1", propertyDefinitions); + expectedPrettyPrint = "LABEL label1 PROPERTIES(propertyA, valueB AS propertyB)"; + assertEquals(expectedPrettyPrint, labelToPropertyDefinitions.prettyPrint()); + + // Test prettyPrint with no properties + labelToPropertyDefinitions = + new GraphElementTable.LabelToPropertyDefinitions("label1", ImmutableList.of()); + expectedPrettyPrint = "LABEL label1 NO PROPERTIES"; + assertEquals(expectedPrettyPrint, labelToPropertyDefinitions.prettyPrint()); + } + + @Test + public void testGraphNodeTableReference() { + GraphElementTable.GraphNodeTableReference graphNodeTableReference = + new GraphElementTable.GraphNodeTableReference( + "nodeTable", ImmutableList.of("nodeKey1", "nodeKey2"), ImmutableList.of("edgeKey")); + + assertEquals("nodeTable", graphNodeTableReference.nodeTableName); + assertEquals(ImmutableList.of("nodeKey1", "nodeKey2"), graphNodeTableReference.nodeKeyColumns); + assertEquals(ImmutableList.of("edgeKey"), graphNodeTableReference.edgeKeyColumns); + + String expectedPrettyPrint = "KEY(edgeKey) REFERENCES nodeTable"; + assertEquals(expectedPrettyPrint, graphNodeTableReference.prettyPrint()); + } + + @Test + public void testGraphElementTable() { + GraphElementTable graphElementTable = + GraphElementTable.builder() + .name("edgeTable") + .baseTableName("baseEdgeTable") + .kind(GraphElementTable.Kind.EDGE) + .keyColumns(ImmutableList.of("edgeKey1", "edgeKey2")) + .labelToPropertyDefinitions( + ImmutableList.of( + new GraphElementTable.LabelToPropertyDefinitions( + "label1", + ImmutableList.of( + new GraphElementTable.PropertyDefinition("propertyA", "valueA"))))) + .sourceNodeTable( + new GraphElementTable.GraphNodeTableReference( + "sourceNodeTable", + ImmutableList.of("sourceNodeKey"), + ImmutableList.of("edgeKey1"))) + .targetNodeTable( + new GraphElementTable.GraphNodeTableReference( + "targetNodeTable", + ImmutableList.of("targetNodeKey"), + ImmutableList.of("edgeKey2"))) + .autoBuild(); + + assertEquals("edgeTable", graphElementTable.name()); + assertEquals("baseEdgeTable", graphElementTable.baseTableName()); + assertEquals(GraphElementTable.Kind.EDGE, graphElementTable.kind()); + assertEquals(Dialect.GOOGLE_STANDARD_SQL, graphElementTable.dialect()); + assertEquals(ImmutableList.of("edgeKey1", "edgeKey2"), graphElementTable.keyColumns()); + + String expectedPrettyPrint = + "baseEdgeTable AS edgeTable\n" + + " KEY (edgeKey1, edgeKey2)\n" + + "SOURCE KEY(edgeKey1) REFERENCES sourceNodeTable DESTINATION KEY(edgeKey2) REFERENCES targetNodeTable\n" + + "LABEL label1 PROPERTIES(valueA AS propertyA)"; + assertEquals(expectedPrettyPrint, graphElementTable.prettyPrint()); + + // Test getLabelToPropertyDefinitions + GraphElementTable.LabelToPropertyDefinitions labelToPropertyDefinitions = + graphElementTable.getLabelToPropertyDefinitions("label1"); + assertEquals("label1", labelToPropertyDefinitions.labelName); + } + + @Test(expected = IllegalArgumentException.class) + public void testPrettyPrintWithInvalidDialect() { + GraphElementTable graphElementTable = + GraphElementTable.builder(Dialect.POSTGRESQL).baseTableName("baseEdgeTable").autoBuild(); + graphElementTable.prettyPrint(); + } +} diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerIT.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerIT.java index 8d0f891603..380e5abf37 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerIT.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/InformationSchemaScannerIT.java @@ -23,7 +23,9 @@ import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; import static org.hamcrest.text.IsEqualCompressingWhiteSpace.equalToCompressingWhiteSpace; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.jupiter.api.Assertions.assertIterableEquals; import com.google.cloud.spanner.BatchClient; import com.google.cloud.spanner.BatchReadOnlyTransaction; @@ -370,6 +372,116 @@ public void simpleModel() throws Exception { assertThat(ddl.prettyPrint(), equalToCompressingWhiteSpace(modelDef)); } + @Test + public void simplePropertyGraph() throws Exception { + String nodeTableDef = + "CREATE TABLE NodeTest (\n" + " Id INT64 NOT NULL,\n" + ") PRIMARY KEY(Id)"; + String edgeTableDef = + "CREATE TABLE EdgeTest (\n" + + "FromId INT64 NOT NULL,\n" + + "ToId INT64 NOT NULL,\n" + + ") PRIMARY KEY(FromId, ToId)"; + String propertyGraphDef = + "CREATE PROPERTY GRAPH testGraph\n" + + " NODE TABLES(\n" + + " NodeTest\n" + + " KEY(Id)\n" + + " LABEL Test PROPERTIES(\n" + + " Id))" + + " EDGE TABLES(\n" + + " EdgeTest\n" + + " KEY(FromId, ToId)\n" + + " SOURCE KEY(FromId) REFERENCES NodeTest(Id)\n" + + " DESTINATION KEY(ToId) REFERENCES NodeTest(Id)\n" + + " DEFAULT LABEL PROPERTIES ALL COLUMNS)"; + + SPANNER_SERVER.createDatabase( + dbId, Arrays.asList(nodeTableDef, edgeTableDef, propertyGraphDef)); + Ddl ddl = getDatabaseDdl(); + + assertThat(ddl.allTables(), hasSize(2)); + assertThat(ddl.table("NodeTest"), notNullValue()); + assertThat(ddl.propertyGraphs(), hasSize(1)); + + PropertyGraph testGraph = ddl.propertyGraph("testGraph"); + + assertEquals(testGraph.name(), "testGraph"); + assertThat(testGraph.propertyDeclarations(), hasSize(3)); + assertThat(testGraph.getPropertyDeclaration("Id"), notNullValue()); + assertThat(testGraph.getPropertyDeclaration("FromId"), notNullValue()); + assertThat(testGraph.getPropertyDeclaration("ToId"), notNullValue()); + + assertThat(testGraph.labels(), hasSize(2)); + assertThat(testGraph.getLabel("Test"), notNullValue()); + assertThat(testGraph.getLabel("EdgeTest"), notNullValue()); + + assertThat(testGraph.nodeTables(), hasSize(1)); + assertThat(testGraph.getNodeTable("NodeTest"), notNullValue()); + + assertThat(testGraph.edgeTables(), hasSize(1)); + assertThat(testGraph.getEdgeTable("EdgeTest"), notNullValue()); + + // --- Assertions for Node Table --- + GraphElementTable nodeTestTable = testGraph.getNodeTable("NodeTest"); + assertThat(nodeTestTable, notNullValue()); + assertThat(nodeTestTable.name(), equalTo("NodeTest")); + assertThat(nodeTestTable.baseTableName(), equalTo("NodeTest")); + assertThat(nodeTestTable.kind(), equalTo(GraphElementTable.Kind.NODE)); + assertIterableEquals(List.of("Id"), nodeTestTable.keyColumns()); + + assertThat(nodeTestTable.labelToPropertyDefinitions(), hasSize(1)); + GraphElementTable.LabelToPropertyDefinitions nodeTestLabel = + nodeTestTable.getLabelToPropertyDefinitions("Test"); + assertThat(nodeTestLabel, notNullValue()); + assertThat(nodeTestLabel.labelName, equalTo("Test")); + assertThat(nodeTestLabel.propertyDefinitions(), hasSize(1)); + GraphElementTable.PropertyDefinition nodeTestIdProperty = + nodeTestLabel.getPropertyDefinition("Id"); + assertThat(nodeTestIdProperty, notNullValue()); + assertThat(nodeTestIdProperty.name, equalTo("Id")); + assertThat(nodeTestIdProperty.valueExpressionString, equalTo("Id")); + + // --- Assertions for Edge Table --- + GraphElementTable edgeTestTable = testGraph.getEdgeTable("EdgeTest"); + assertThat(edgeTestTable, notNullValue()); + assertThat(edgeTestTable.name(), equalTo("EdgeTest")); + assertThat(edgeTestTable.baseTableName(), equalTo("EdgeTest")); + assertThat(edgeTestTable.kind(), equalTo(GraphElementTable.Kind.EDGE)); + assertIterableEquals(List.of("FromId", "ToId"), edgeTestTable.keyColumns()); + + assertThat(edgeTestTable.labelToPropertyDefinitions(), hasSize(1)); + GraphElementTable.LabelToPropertyDefinitions edgeTestLabel = + edgeTestTable.getLabelToPropertyDefinitions("EdgeTest"); + assertThat(edgeTestLabel, notNullValue()); + assertThat(edgeTestLabel.labelName, equalTo("EdgeTest")); + assertThat(edgeTestLabel.propertyDefinitions(), hasSize(2)); // FromId and ToId + + GraphElementTable.PropertyDefinition edgeTestFromIdProperty = + edgeTestLabel.getPropertyDefinition("FromId"); + assertThat(edgeTestFromIdProperty, notNullValue()); + assertThat(edgeTestFromIdProperty.name, equalTo("FromId")); + assertThat(edgeTestFromIdProperty.valueExpressionString, equalTo("FromId")); + + GraphElementTable.PropertyDefinition edgeTestToIdProperty = + edgeTestLabel.getPropertyDefinition("ToId"); + assertThat(edgeTestToIdProperty, notNullValue()); + assertThat(edgeTestToIdProperty.name, equalTo("ToId")); + assertThat(edgeTestToIdProperty.valueExpressionString, equalTo("ToId")); + + // --- Assertions for Edge Table References --- + assertThat(edgeTestTable.sourceNodeTable().nodeTableName, equalTo("NodeTest")); + assertIterableEquals(List.of("Id"), edgeTestTable.sourceNodeTable().nodeKeyColumns); + + assertIterableEquals(List.of("FromId"), edgeTestTable.sourceNodeTable().edgeKeyColumns); + + assertThat(edgeTestTable.targetNodeTable().nodeTableName, equalTo("NodeTest")); + assertIterableEquals(List.of("Id"), edgeTestTable.targetNodeTable().nodeKeyColumns); + assertIterableEquals(List.of("ToId"), edgeTestTable.targetNodeTable().edgeKeyColumns); + } + + @Test + public void complexPropertyGraph() throws Exception {} + @Test public void simpleView() throws Exception { String tableDef = diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/PropertyGraphTest.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/PropertyGraphTest.java new file mode 100644 index 0000000000..3669df2f7d --- /dev/null +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ddl/PropertyGraphTest.java @@ -0,0 +1,205 @@ +/* + * Copyright (C) 2024 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.google.cloud.spanner.Dialect; +import com.google.common.collect.ImmutableList; +import org.junit.Test; + +public class PropertyGraphTest { + + @Test + public void testPropertyDeclaration() { + PropertyGraph.PropertyDeclaration propertyDeclaration = + new PropertyGraph.PropertyDeclaration("propertyA", "INT64"); + assertEquals("propertyA", propertyDeclaration.name); + assertEquals("INT64", propertyDeclaration.type); + } + + @Test + public void testGraphElementLabel() { + PropertyGraph.GraphElementLabel graphElementLabel = + new PropertyGraph.GraphElementLabel("label1", ImmutableList.of("propertyA", "propertyB")); + assertEquals("label1", graphElementLabel.name); + assertEquals(ImmutableList.of("propertyA", "propertyB"), graphElementLabel.properties); + } + + @Test + public void testPropertyGraph() { + GraphElementTable nodeTable = + GraphElementTable.builder() + .name("nodeTable") + .baseTableName("baseNodeTable") + .kind(GraphElementTable.Kind.NODE) + .keyColumns(ImmutableList.of("nodeKey")) + .labelToPropertyDefinitions( + ImmutableList.of( + new GraphElementTable.LabelToPropertyDefinitions( + "nodeLabel", + ImmutableList.of( + new GraphElementTable.PropertyDefinition("propertyA", "valueA"))))) + .autoBuild(); + + GraphElementTable edgeTable = + GraphElementTable.builder() + .name("edgeTable") + .baseTableName("baseEdgeTable") + .kind(GraphElementTable.Kind.EDGE) + .keyColumns(ImmutableList.of("edgeKey1", "edgeKey2")) + .labelToPropertyDefinitions( + ImmutableList.of( + new GraphElementTable.LabelToPropertyDefinitions( + "edgeLabel", + ImmutableList.of( + new GraphElementTable.PropertyDefinition("propertyB", "valueB"))))) + .sourceNodeTable( + new GraphElementTable.GraphNodeTableReference( + "nodeTable", ImmutableList.of("nodeKey"), ImmutableList.of("edgeKey1"))) + .targetNodeTable( + new GraphElementTable.GraphNodeTableReference( + "nodeTable", ImmutableList.of("nodeKey"), ImmutableList.of("edgeKey2"))) + .autoBuild(); + + PropertyGraph propertyGraph = + PropertyGraph.builder() + .name("myGraph") + .addNodeTable(nodeTable) + .addEdgeTable(edgeTable) + .addPropertyDeclaration(new PropertyGraph.PropertyDeclaration("propertyA", "INT64")) + .addLabel( + new PropertyGraph.GraphElementLabel("nodeLabel", ImmutableList.of("propertyA"))) + .build(); + + assertEquals("myGraph", propertyGraph.name()); + assertEquals(ImmutableList.of(nodeTable), propertyGraph.nodeTables()); + assertEquals(ImmutableList.of(edgeTable), propertyGraph.edgeTables()); + + // Test getPropertyDeclaration + PropertyGraph.PropertyDeclaration propertyDeclaration = + propertyGraph.getPropertyDeclaration("propertyA"); + assertEquals("propertyA", propertyDeclaration.name); + + // Test getLabel + PropertyGraph.GraphElementLabel label = propertyGraph.getLabel("nodeLabel"); + assertEquals("nodeLabel", label.name); + + // Test getNodeTable + GraphElementTable retrievedNodeTable = propertyGraph.getNodeTable("nodeTable"); + assertEquals("nodeTable", retrievedNodeTable.name()); + + // Test getEdgeTable + GraphElementTable retrievedEdgeTable = propertyGraph.getEdgeTable("edgeTable"); + assertEquals("edgeTable", retrievedEdgeTable.name()); + + String expectedPrettyPrint = + "CREATE PROPERTY GRAPH myGraph\n" + + "NODE TABLES(\n" + + "baseNodeTable AS nodeTable\n" + + " KEY (nodeKey)\n" + + "LABEL nodeLabel PROPERTIES(valueA AS propertyA)" + + ")\n" + + "EDGE TABLES(\n" + + "baseEdgeTable AS edgeTable\n" + + " KEY (edgeKey1, edgeKey2)\n" + + "SOURCE KEY(edgeKey1) REFERENCES nodeTable DESTINATION KEY(edgeKey2) REFERENCES nodeTable\n" + + "LABEL edgeLabel PROPERTIES(valueB AS propertyB)" + + ")"; + assertEquals(expectedPrettyPrint, propertyGraph.prettyPrint()); + } + + @Test + public void testPropertyGraphBuilder() { + PropertyGraph.Builder builder = PropertyGraph.builder(); + builder.name("myGraph"); + builder.dialect(Dialect.GOOGLE_STANDARD_SQL); + + GraphElementTable nodeTable = + GraphElementTable.builder(builder.dialect()) + .name("nodeTable") + .baseTableName("baseNodeTable") + .kind(GraphElementTable.Kind.NODE) + .keyColumns(ImmutableList.of("nodeKey")) + .autoBuild(); + + GraphElementTable edgeTable = + GraphElementTable.builder(builder.dialect()) + .name("edgeTable") + .baseTableName("baseEdgeTable") + .kind(GraphElementTable.Kind.EDGE) + .keyColumns(ImmutableList.of("edgeKey1", "edgeKey2")) + .sourceNodeTable( + new GraphElementTable.GraphNodeTableReference( + "nodeTable", ImmutableList.of("nodeKey"), ImmutableList.of("edgeKey1"))) + .targetNodeTable( + new GraphElementTable.GraphNodeTableReference( + "nodeTable", ImmutableList.of("nodeKey"), ImmutableList.of("edgeKey2"))) + .autoBuild(); + + builder.addNodeTable(nodeTable); + builder.addEdgeTable(edgeTable); + builder.addPropertyDeclaration(new PropertyGraph.PropertyDeclaration("propertyA", "INT64")); + builder.addLabel( + new PropertyGraph.GraphElementLabel("nodeLabel", ImmutableList.of("propertyA"))); + + PropertyGraph propertyGraph = builder.build(); + + assertEquals("myGraph", propertyGraph.name()); + assertEquals(1, propertyGraph.nodeTables().size()); + assertEquals(nodeTable, propertyGraph.nodeTables().get(0)); + assertEquals(1, propertyGraph.edgeTables().size()); + assertEquals(edgeTable, propertyGraph.edgeTables().get(0)); + } + + @Test + public void testToBuilder() { + PropertyGraph propertyGraph = + PropertyGraph.builder() + .name("myGraph") + .addNodeTable( + GraphElementTable.builder() + .name("nodeTable") + .baseTableName("baseNodeTable") + .kind(GraphElementTable.Kind.NODE) + .keyColumns(ImmutableList.of("nodeKey")) + .autoBuild()) + .addPropertyDeclaration(new PropertyGraph.PropertyDeclaration("propertyA", "INT64")) + .addLabel( + new PropertyGraph.GraphElementLabel("nodeLabel", ImmutableList.of("propertyA"))) + .build(); + + PropertyGraph.Builder builder = propertyGraph.toBuilder(); + PropertyGraph newPropertyGraph = builder.build(); + + assertEquals(propertyGraph.name(), newPropertyGraph.name()); + assertEquals(propertyGraph.nodeTables(), newPropertyGraph.nodeTables()); + assertEquals(propertyGraph.edgeTables(), newPropertyGraph.edgeTables()); + assertEquals(propertyGraph.propertyDeclarations(), newPropertyGraph.propertyDeclarations()); + assertEquals(propertyGraph.labels(), newPropertyGraph.labels()); + } + + @Test + public void testGettersWithEmptyLists() { + PropertyGraph propertyGraph = PropertyGraph.builder().name("myGraph").build(); + + assertNull(propertyGraph.getPropertyDeclaration("propertyA")); + assertNull(propertyGraph.getLabel("nodeLabel")); + assertNull(propertyGraph.getNodeTable("nodeTable")); + assertNull(propertyGraph.getEdgeTable("edgeTable")); + } +} From 28653c61227dc524aefb1a50b2e2db066444d829 Mon Sep 17 00:00:00 2001 From: Vardhan Vinay Thigle <39047439+VardhanThigle@users.noreply.github.com> Date: Mon, 2 Dec 2024 14:49:51 +0000 Subject: [PATCH 06/14] Setting useCursorFetch for MySql if Fetchsize is configured. (#2041) --- .../v2/options/OptionsToConfigBuilder.java | 28 +++++++++++++++++++ .../v2/options/SourceDbToSpannerOptions.java | 5 ++-- .../options/OptionsToConfigBuilderTest.java | 16 +++++++++++ 3 files changed, 47 insertions(+), 2 deletions(-) diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java index 57f12abc2d..9f75fc9f0c 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java @@ -32,6 +32,7 @@ import java.net.URISyntaxException; import java.util.List; import java.util.Map.Entry; +import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.Wait; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -128,6 +129,7 @@ public static JdbcIOWrapperConfig getJdbcIOWrapperConfig( MySqlConfigDefaults.DEFAULT_MYSQL_URL_PROPERTIES.entrySet()) { sourceDbURL = addParamToJdbcUrl(sourceDbURL, entry.getKey(), entry.getValue()); } + sourceDbURL = mysqlSetCursorModeIfNeeded(sqlDialect, sourceDbURL, fetchSize); break; case POSTGRESQL: if (sourceDbURL == null) { @@ -155,6 +157,32 @@ public static JdbcIOWrapperConfig getJdbcIOWrapperConfig( return builder.build(); } + /** + * For MySQL Dialect, if Fetchsize is expecitly set by the user, enables `useCursorFetch`. + * + * @param sqlDialect Sql Dialect. + * @param url DB Url from passed configs. + * @param fetchSize FetchSize Setting (Null if user has not explicitly set) + * @return Updated URL with `useCursorFetch` only if dialect is MySql and Fetchsize is not null. + * Same as input URL in all other cases. + */ + @VisibleForTesting + @Nullable + protected static String mysqlSetCursorModeIfNeeded( + SQLDialect sqlDialect, String url, @Nullable Integer fetchSize) { + if (fetchSize == null) { + LOG.info( + "FetchSize is not explicitly configured. In case of out of memory errors, please set `FetSize` according to the available memory and maximum size of a row."); + return url; + } + if (sqlDialect != SQLDialect.MYSQL) { + return url; + } + LOG.info("For Mysql, Fetchsize is explicitly configured. So setting `useCursorMode=true`."); + String updatedUrl = addParamToJdbcUrl(url, "useCursorFetch", "true"); + return updatedUrl; + } + @VisibleForTesting protected static String addParamToJdbcUrl(String jdbcUrl, String paramName, String paramValue) { // URI/ URL libraries don't seem to handle jdbc URLs well diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java index 428010f91e..5a58bb61e2 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/SourceDbToSpannerOptions.java @@ -122,9 +122,10 @@ public interface SourceDbToSpannerOptions extends CommonTemplateOptions { optional = true, description = "The number of rows to fetch per page read for JDBC source.", helpText = - "The number of rows to fetch per page read for JDBC source. If not set, the default of JdbcIO of 50_000 rows gets used." + "The number of rows to fetch per page read for JDBC source. If not set, the default of JdbcIO of 50_000 rows gets used. If source dialect is Mysql, please see the note below." + " This ultimately translated to Statement.setFetchSize call at Jdbc layer. It should ONLY be used if the default value throws memory errors." - + "In case you are using MySql source, please also note that FetchSize is ignored by the connector unless, you also set `useCursorFetch=true` as a connection property either in the source URL or the shard config file") + + "Note for MySql Source: FetchSize is ignored by the Mysql connector unless, `useCursorFetch=true` is also part of the connection properties." + + "In case, the fetchSize parameter is explicitly set, for MySql dialect, the pipeline will add `useCursorFetch=true` to the connection properties by default.") Integer getFetchSize(); void setFetchSize(Integer value); diff --git a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java index 9ea7010288..d070d6b829 100644 --- a/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java +++ b/v2/sourcedb-to-spanner/src/test/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilderTest.java @@ -287,4 +287,20 @@ public void testaddParamToJdbcUrl() throws URISyntaxException { "allowMultiQueries", "true")); } + + @Test + public void testMySqlSetCursorModeIfNeeded() { + assertThat( + OptionsToConfigBuilder.mysqlSetCursorModeIfNeeded( + SQLDialect.MYSQL, "jdbc:mysql://localhost:3306/testDB?useSSL=true", 42)) + .isEqualTo("jdbc:mysql://localhost:3306/testDB?useSSL=true&useCursorFetch=true"); + assertThat( + OptionsToConfigBuilder.mysqlSetCursorModeIfNeeded( + SQLDialect.MYSQL, "jdbc:mysql://localhost:3306/testDB?useSSL=true", null)) + .isEqualTo("jdbc:mysql://localhost:3306/testDB?useSSL=true"); + assertThat( + OptionsToConfigBuilder.mysqlSetCursorModeIfNeeded( + SQLDialect.POSTGRESQL, "jdbc:mysql://localhost:3306/testDB?useSSL=true", 42)) + .isEqualTo("jdbc:mysql://localhost:3306/testDB?useSSL=true"); + } } From 7be86378abd37e9e107e4aafb4b9470f8ef2177a Mon Sep 17 00:00:00 2001 From: Jeff Kinard Date: Mon, 2 Dec 2024 13:37:32 -0500 Subject: [PATCH 07/14] Add DLQ support to JdbcToBigQuery using BQ Storage Write API (#2016) * Add DLQ support to JdbcToBigQuery using BQ Storage Write API Signed-off-by: Jeffrey Kinard * fix test failures Signed-off-by: Jeffrey Kinard --------- Signed-off-by: Jeffrey Kinard --- .../v2/transforms/ErrorConverters.java | 26 +++++- .../v2/options/JdbcToBigQueryOptions.java | 15 +++ .../teleport/v2/templates/JdbcToBigQuery.java | 92 ++++++++++++++++++- .../v2/templates/JdbcToBigQueryIT.java | 71 ++++++++++++-- 4 files changed, 194 insertions(+), 10 deletions(-) diff --git a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/ErrorConverters.java b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/ErrorConverters.java index 49466183b7..9f6b72a525 100644 --- a/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/ErrorConverters.java +++ b/v2/common/src/main/java/com/google/cloud/teleport/v2/transforms/ErrorConverters.java @@ -59,6 +59,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Ascii; import org.joda.time.DateTimeZone; import org.joda.time.Duration; +import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @@ -76,18 +77,23 @@ public abstract static class WriteStringMessageErrors extends PTransform>, WriteResult> { public static Builder newBuilder() { - return new AutoValue_ErrorConverters_WriteStringMessageErrors.Builder(); + return new AutoValue_ErrorConverters_WriteStringMessageErrors.Builder() + .setUseWindowedTimestamp(true); } public abstract String getErrorRecordsTable(); public abstract String getErrorRecordsTableSchema(); + public abstract boolean getUseWindowedTimestamp(); + @Override public WriteResult expand(PCollection> failedRecords) { return failedRecords - .apply("FailedRecordToTableRow", ParDo.of(new FailedStringToTableRowFn())) + .apply( + "FailedRecordToTableRow", + ParDo.of(new FailedStringToTableRowFn(getUseWindowedTimestamp()))) .apply( "WriteFailedRecordsToBigQuery", BigQueryIO.writeTableRows() @@ -104,6 +110,8 @@ public abstract static class Builder { public abstract Builder setErrorRecordsTableSchema(String errorRecordsTableSchema); + public abstract Builder setUseWindowedTimestamp(boolean useWindowedTimestamp); + public abstract WriteStringMessageErrors build(); } } @@ -115,6 +123,16 @@ public abstract static class Builder { public static class FailedStringToTableRowFn extends DoFn, TableRow> { + private boolean useWindowedTimestamp; + + public FailedStringToTableRowFn() { + this(true); + } + + public FailedStringToTableRowFn(boolean useWindowedTimestamp) { + this.useWindowedTimestamp = useWindowedTimestamp; + } + /** * The formatter used to convert timestamps into a BigQuery compatible format. @@ -129,7 +147,9 @@ public void processElement(ProcessContext context) { // Format the timestamp for insertion String timestamp = - TIMESTAMP_FORMATTER.print(context.timestamp().toDateTime(DateTimeZone.UTC)); + TIMESTAMP_FORMATTER.print( + (useWindowedTimestamp ? context.timestamp() : Instant.now()) + .toDateTime(DateTimeZone.UTC)); // Build the table row TableRow failedRow = diff --git a/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/JdbcToBigQueryOptions.java b/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/JdbcToBigQueryOptions.java index 5fd75bb9f1..8f2769b483 100644 --- a/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/JdbcToBigQueryOptions.java +++ b/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/JdbcToBigQueryOptions.java @@ -256,4 +256,19 @@ public interface JdbcToBigQueryOptions String getBigQuerySchemaPath(); void setBigQuerySchemaPath(String path); + + @TemplateParameter.BigQueryTable( + order = 21, + optional = true, + description = + "Table for messages that failed to reach the output table (i.e., Deadletter table) when using Storage Write API", + helpText = + "The BigQuery table to use for messages that failed to reach the output table, " + + "formatted as `\"PROJECT_ID:DATASET_NAME.TABLE_NAME\"`. If the table " + + "doesn't exist, it is created when the pipeline runs. " + + "If this parameter is not specified, the pipeline will fail on write errors." + + "This parameter can only be specified if `useStorageWriteApi` or `useStorageWriteApiAtLeastOnce` is set to true.") + String getOutputDeadletterTable(); + + void setOutputDeadletterTable(String value); } diff --git a/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToBigQuery.java b/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToBigQuery.java index ce34c3e0ac..04de8da1de 100644 --- a/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToBigQuery.java +++ b/v2/jdbc-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/JdbcToBigQuery.java @@ -15,28 +15,45 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.transforms.BigQueryConverters.wrapBigQueryInsertError; import static com.google.cloud.teleport.v2.utils.GCSUtils.getGcsFileAsString; import static com.google.cloud.teleport.v2.utils.KMSUtils.maybeDecrypt; +import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.teleport.metadata.Template; import com.google.cloud.teleport.metadata.TemplateCategory; +import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; import com.google.cloud.teleport.v2.common.UncaughtExceptionLogger; import com.google.cloud.teleport.v2.options.JdbcToBigQueryOptions; +import com.google.cloud.teleport.v2.transforms.ErrorConverters; import com.google.cloud.teleport.v2.utils.BigQueryIOUtils; import com.google.cloud.teleport.v2.utils.GCSAwareValueProvider; import com.google.cloud.teleport.v2.utils.JdbcConverters; +import com.google.cloud.teleport.v2.utils.ResourceUtils; import com.google.cloud.teleport.v2.utils.SecretManagerUtils; +import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; 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.FileSystems; 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.BigQueryInsertError; +import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy; import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; +import org.apache.beam.sdk.io.gcp.bigquery.WriteResult; import org.apache.beam.sdk.io.jdbc.JdbcIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; /** @@ -69,6 +86,10 @@ }) public class JdbcToBigQuery { + /** Coder for FailsafeElement. */ + private static final FailsafeElementCoder FAILSAFE_ELEMENT_CODER = + FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + /** * Main entry point for executing the pipeline. This will run the pipeline asynchronously. If * blocking execution is required, use the {@link JdbcToBigQuery#run} method to start the pipeline @@ -97,6 +118,12 @@ public static void main(String[] args) { static PipelineResult run(JdbcToBigQueryOptions options, Write writeToBQ) { // Validate BQ STORAGE_WRITE_API options BigQueryIOUtils.validateBQStorageApiOptionsBatch(options); + if (!options.getUseStorageWriteApi() + && !options.getUseStorageWriteApiAtLeastOnce() + && !Strings.isNullOrEmpty(options.getOutputDeadletterTable())) { + throw new IllegalArgumentException( + "outputDeadletterTable can only be specified if BigQuery Storage Write API is enabled either with useStorageWriteApi or useStorageWriteApiAtLeastOnce."); + } // Create the pipeline Pipeline pipeline = Pipeline.create(options); @@ -175,12 +202,71 @@ static PipelineResult run(JdbcToBigQueryOptions options, Write writeTo /* * Step 2: Append TableRow to an existing BigQuery table */ - rows.apply("Write to BigQuery", writeToBQ); + WriteResult writeResult = rows.apply("Write to BigQuery", writeToBQ); + + /* + * Step 3. + * If using Storage Write API, capture failed inserts and either + * a) write error rows to DLQ + * b) fail the pipeline + */ + if (options.getUseStorageWriteApi() || options.getUseStorageWriteApiAtLeastOnce()) { + PCollection insertErrors = + BigQueryIOUtils.writeResultToBigQueryInsertErrors(writeResult, options); + + if (!Strings.isNullOrEmpty(options.getOutputDeadletterTable())) { + /* + * Step 3a. + * Elements that failed inserts into BigQuery are extracted and converted to FailsafeElement + */ + PCollection> failedInserts = + insertErrors + .apply( + "WrapInsertionErrors", + MapElements.into(FAILSAFE_ELEMENT_CODER.getEncodedTypeDescriptor()) + .via((BigQueryInsertError e) -> wrapBigQueryInsertError(e))) + .setCoder(FAILSAFE_ELEMENT_CODER); + + /* + * Step 3a Contd. + * Insert records that failed insert into deadletter table + */ + failedInserts.apply( + "WriteFailedRecords", + ErrorConverters.WriteStringMessageErrors.newBuilder() + .setErrorRecordsTable(options.getOutputDeadletterTable()) + .setErrorRecordsTableSchema(ResourceUtils.getDeadletterTableSchemaJson()) + .setUseWindowedTimestamp(false) + .build()); + } else { + /* + * Step 3b. + * Fail pipeline upon write errors if no DLQ was specified + */ + insertErrors.apply(ParDo.of(new ThrowWriteErrorsDoFn())); + } + } // Execute the pipeline and return the result. return pipeline.run(); } + static class ThrowWriteErrorsDoFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + BigQueryInsertError insertError = Objects.requireNonNull(c.element()); + List errorMessages = + insertError.getError().getErrors().stream() + .map(ErrorProto::getMessage) + .collect(Collectors.toList()); + String stackTrace = String.join("\nCaused by:", errorMessages); + + throw new IllegalStateException( + String.format( + "Failed to insert row %s.\nCaused by: %s", insertError.getRow(), stackTrace)); + } + } + /** * Create the {@link Write} transform that outputs the collection to BigQuery as per input option. */ @@ -198,12 +284,16 @@ static Write writeToBQTransform(JdbcToBigQueryOptions options) { : Write.WriteDisposition.WRITE_APPEND) .withCustomGcsTempLocation( StaticValueProvider.of(options.getBigQueryLoadingTemporaryDirectory())) + .withExtendedErrorInfo() .to(options.getOutputTable()); if (Write.CreateDisposition.valueOf(options.getCreateDisposition()) != Write.CreateDisposition.CREATE_NEVER) { write = write.withJsonSchema(getGcsFileAsString(options.getBigQuerySchemaPath())); } + if (options.getUseStorageWriteApi() || options.getUseStorageWriteApiAtLeastOnce()) { + write = write.withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors()); + } return write; } diff --git a/v2/jdbc-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToBigQueryIT.java b/v2/jdbc-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToBigQueryIT.java index 3f8d0f5fc0..f95475c0e9 100644 --- a/v2/jdbc-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToBigQueryIT.java +++ b/v2/jdbc-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/JdbcToBigQueryIT.java @@ -66,6 +66,8 @@ public class JdbcToBigQueryIT extends JDBCBaseIT { private static final Logger LOG = LoggerFactory.getLogger(JdbcToBigQueryIT.class); + private static final Integer NUM_ROWS = 100; + private static final String ROW_ID = "row_id"; private static final String NAME = "name"; private static final String FULL_NAME = "full_name"; @@ -73,6 +75,7 @@ public class JdbcToBigQueryIT extends JDBCBaseIT { private static final String MEMBER = "member"; private static final String IS_MEMBER = "is_member"; private static final String ENTRY_ADDED = "entry_added"; + private static final String FAKE = "FAKE"; private static final String KMS_REGION = "global"; private static final String KEYRING_ID = "JDBCToBigQuery"; @@ -125,6 +128,7 @@ public void testMySqlToBigQueryFlex() throws IOException { mySqlDriverGCSPath(), mySQLResourceManager, true, + false, config -> config.addParameter( "query", @@ -132,6 +136,36 @@ public void testMySqlToBigQueryFlex() throws IOException { + testName)); } + @Test + public void testMySqlToBigQueryFlexWithDlq() throws IOException { + // Create MySQL Resource manager + mySQLResourceManager = MySQLResourceManager.builder(testName).build(); + + // Arrange MySQL-compatible schema + HashMap columns = new HashMap<>(); + columns.put(ROW_ID, "NUMERIC NOT NULL"); + columns.put(NAME, "VARCHAR(200)"); + columns.put(AGE, "NUMERIC"); + columns.put(MEMBER, "VARCHAR(200)"); + columns.put(ENTRY_ADDED, "VARCHAR(200)"); + columns.put(FAKE, "VARCHAR(200)"); + JDBCResourceManager.JDBCSchema schema = new JDBCResourceManager.JDBCSchema(columns, ROW_ID); + + // Run a simple IT + simpleJdbcToBigQueryTest( + testName, + schema, + MYSQL_DRIVER, + mySqlDriverGCSPath(), + mySQLResourceManager, + true, + true, + config -> + config + .addParameter("query", "select * from " + testName) + .addParameter("useStorageWriteApi", "true")); + } + @Test public void testMySqlToBigQueryWithStorageWriteApi() throws IOException { // Create MySQL Resource manager @@ -156,6 +190,7 @@ public void testMySqlToBigQueryWithStorageWriteApi() throws IOException { mySqlDriverGCSPath(), mySQLResourceManager, true, + false, config -> config .addParameter( @@ -186,6 +221,7 @@ public void testPostgresToBigQueryFlex() throws IOException { postgresDriverGCSPath(), postgresResourceManager, true, + false, config -> config.addParameter( "query", @@ -220,6 +256,7 @@ public void testPostgresWithUnicodeCharactersInQuery() throws IOException { postgresDriverGCSPath(), postgresResourceManager, true, + false, config -> config.addParameter("query", getGcsPath("input/query.sql"))); } @@ -251,6 +288,7 @@ public void testOracleToBigQueryFlex() throws IOException { oracleDriverGCSPath(), oracleResourceManager, true, + false, config -> config.addParameter( "query", @@ -280,6 +318,7 @@ public void testMsSqlToBigQueryFlex() throws IOException { msSqlDriverGCSPath(), msSQLResourceManager, true, + false, config -> config.addParameter( "query", @@ -307,6 +346,7 @@ public void testReadWithPartitions() throws IOException { postgresDriverGCSPath(), postgresResourceManager, false, + false, config -> config.addParameter("table", testName).addParameter("partitionColumn", ROW_ID)); } @@ -317,6 +357,7 @@ private void simpleJdbcToBigQueryTest( String driverJars, JDBCResourceManager jdbcResourceManager, boolean useColumnAlias, + boolean useDlq, Function paramsAdder) throws IOException { simpleJdbcToBigQueryTest( @@ -328,6 +369,7 @@ private void simpleJdbcToBigQueryTest( driverJars, jdbcResourceManager, useColumnAlias, + useDlq, paramsAdder); } @@ -340,12 +382,16 @@ private void simpleJdbcToBigQueryTest( String driverJars, JDBCResourceManager jdbcResourceManager, boolean useColumnAlias, + boolean useDlq, Function paramsAdder) throws IOException { // Arrange - List> jdbcData = - getJdbcData(List.of(ROW_ID, NAME, AGE, MEMBER, ENTRY_ADDED)); + List columns = new ArrayList<>(List.of(ROW_ID, NAME, AGE, MEMBER, ENTRY_ADDED)); + if (useDlq) { + columns.add(FAKE); + } + List> jdbcData = getJdbcData(columns, useDlq); jdbcResourceManager.createTable(sourceTableName, schema); jdbcResourceManager.write(sourceTableName, jdbcData); @@ -380,6 +426,9 @@ private void simpleJdbcToBigQueryTest( .addParameter("fetchSize", "100000") .addParameter("connectionProperties", "characterEncoding=UTF-8") .addParameter("disabledAlgorithms", "SSLv3, GCM")); + if (useDlq) { + options.addParameter("outputDeadletterTable", toTableSpecLegacy(table) + "_error_records"); + } // Act PipelineLauncher.LaunchInfo info = launchTemplate(options); @@ -397,8 +446,15 @@ private void simpleJdbcToBigQueryTest( row.put("is_member", row.remove("member")); }); } - assertThatBigQueryRecords(bigQueryResourceManager.readTable(targetTableName)) - .hasRecordsUnorderedCaseInsensitiveColumns(jdbcData); + if (useDlq) { + assertThatBigQueryRecords(bigQueryResourceManager.readTable(targetTableName)).hasRows(0); + assertThatBigQueryRecords( + bigQueryResourceManager.readTable(targetTableName + "_error_records")) + .hasRows(NUM_ROWS); + } else { + assertThatBigQueryRecords(bigQueryResourceManager.readTable(targetTableName)) + .hasRecordsUnorderedCaseInsensitiveColumns(jdbcData); + } } /** @@ -407,15 +463,18 @@ private void simpleJdbcToBigQueryTest( * @param columns List of column names. * @return A map containing the rows of data to be stored in each JDBC table. */ - private List> getJdbcData(List columns) { + private List> getJdbcData(List columns, boolean useDlq) { List> data = new ArrayList<>(); - for (int i = 0; i < 100; i++) { + for (int i = 0; i < NUM_ROWS; i++) { Map values = new HashMap<>(); values.put(columns.get(0), i); values.put(columns.get(1), RandomStringUtils.randomAlphabetic(10)); values.put(columns.get(2), new Random().nextInt(100)); values.put(columns.get(3), i % 2 == 0 ? "Y" : "N"); values.put(columns.get(4), Instant.now().toString()); + if (useDlq) { + values.put(columns.get(5), RandomStringUtils.randomAlphabetic(10)); + } data.add(values); } From 1c29788eea56bb55d41db7b002edbfcd5b6ade20 Mon Sep 17 00:00:00 2001 From: Nick Anikin <52892974+an2x@users.noreply.github.com> Date: Mon, 2 Dec 2024 16:16:10 -0500 Subject: [PATCH 08/14] [KafkaToBigQueryFlex Template]: Add support for Avro ENUM type and fix FLOAT type. (#2029) Fix 2 issues: 1. Using Avro format with an ENUM type field in the KafkaToBigQueryFlex template currently causes the following error: "Expected Avro schema types [STRING] for BigQuery STRING field operation, but received ENUM". 2. Avro FLOAT type also wasn't handled correctly, resulting in the following error: "Expected Avro schema types [DOUBLE, INT] for BigQuery FLOAT64 field <...>, but received FLOAT". --- .../teleport/v2/utils/BigQueryAvroUtils.java | 24 +++-- .../templates/KafkaToBigQueryFlexAvroIT.java | 87 +++++++++++++------ .../avro_schema.avsc | 18 ++++ 3 files changed, 99 insertions(+), 30 deletions(-) diff --git a/v2/kafka-to-bigquery/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryAvroUtils.java b/v2/kafka-to-bigquery/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryAvroUtils.java index 9150f378bb..08558e1f38 100644 --- a/v2/kafka-to-bigquery/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryAvroUtils.java +++ b/v2/kafka-to-bigquery/src/main/java/com/google/cloud/teleport/v2/utils/BigQueryAvroUtils.java @@ -46,6 +46,7 @@ import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.Schema.Type; +import org.apache.avro.generic.GenericData.EnumSymbol; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; @@ -77,15 +78,18 @@ public class BigQueryAvroUtils { static final ImmutableMultimap BIG_QUERY_TO_AVRO_TYPES = ImmutableMultimap.builder() .put("STRING", Type.STRING) + .put("STRING", Type.ENUM) .put("GEOGRAPHY", Type.STRING) .put("BYTES", Type.BYTES) .put("INTEGER", Type.LONG) .put("INTEGER", Type.INT) .put("INT64", Type.LONG) .put("INT64", Type.INT) - .put("FLOAT", Type.DOUBLE) - .put("FLOAT", Type.INT) + // BigQueryUtils map both Avro's FLOAT and DOUBLE to BigQuery's FLOAT64: + // https://github.com/apache/beam/blob/b70375db84a7ff04a6be3aea8d5ae30f4d7cdbe1/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java#L227-L228 .put("FLOAT64", Type.DOUBLE) + .put("FLOAT64", Type.FLOAT) + .put("FLOAT64", Type.LONG) .put("FLOAT64", Type.INT) .put("NUMERIC", Type.BYTES) .put("BIGNUMERIC", Type.BYTES) @@ -312,11 +316,16 @@ private static Object convertRequiredField( // BigQuery represents NUMERIC in Avro format as BYTES with a DECIMAL logical type. switch (bqType) { case "STRING": + // Avro will use a CharSequence to represent String objects, but it may not always use + // java.lang.String; for example, it may prefer org.apache.avro.util.Utf8. + verify( + v instanceof CharSequence || v instanceof EnumSymbol, + "Expected CharSequence (String) or EnumSymbol, got %s", + v.getClass()); + return v.toString(); case "DATETIME": case "GEOGRAPHY": case "JSON": - // Avro will use a CharSequence to represent String objects, but it may not always use - // java.lang.String; for example, it may prefer org.apache.avro.util.Utf8. verify(v instanceof CharSequence, "Expected CharSequence (String), got %s", v.getClass()); return v.toString(); case "DATE": @@ -350,11 +359,16 @@ private static Object convertRequiredField( verify(v instanceof Long, "Expected Long, got %s", v.getClass()); return ((Long) v).toString(); } - case "FLOAT": case "FLOAT64": if (avroType == Type.INT) { verify(v instanceof Integer, "Expected Integer, got %s", v.getClass()); return (Double) ((Integer) v).doubleValue(); + } else if (avroType == Type.LONG) { + verify(v instanceof Long, "Expected Long, got %s", v.getClass()); + return (Double) ((Long) v).doubleValue(); + } else if (avroType == Type.FLOAT) { + verify(v instanceof Float, "Expected Float, got %s", v.getClass()); + return (Double) ((Float) v).doubleValue(); } else { verify(v instanceof Double, "Expected Double, got %s", v.getClass()); return v; diff --git a/v2/kafka-to-bigquery/src/test/java/com/google/cloud/teleport/v2/templates/KafkaToBigQueryFlexAvroIT.java b/v2/kafka-to-bigquery/src/test/java/com/google/cloud/teleport/v2/templates/KafkaToBigQueryFlexAvroIT.java index 9be7504cef..7ecb444edf 100644 --- a/v2/kafka-to-bigquery/src/test/java/com/google/cloud/teleport/v2/templates/KafkaToBigQueryFlexAvroIT.java +++ b/v2/kafka-to-bigquery/src/test/java/com/google/cloud/teleport/v2/templates/KafkaToBigQueryFlexAvroIT.java @@ -40,6 +40,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import net.jcip.annotations.NotThreadSafe; +import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.it.common.PipelineLauncher.LaunchConfig; @@ -81,6 +82,7 @@ public final class KafkaToBigQueryFlexAvroIT extends TemplateTestBase { private TableId tableId; private Schema bqSchema; private org.apache.avro.Schema avroSchema; + private org.apache.avro.Schema avroSchemaUsageEnum; private org.apache.avro.Schema otherAvroSchema; @Before @@ -90,7 +92,9 @@ public void setup() throws IOException { bqSchema = Schema.of( Field.of("productId", StandardSQLTypeName.INT64), - Field.newBuilder("productName", StandardSQLTypeName.STRING).setMaxLength(10L).build()); + Field.newBuilder("productName", StandardSQLTypeName.STRING).setMaxLength(10L).build(), + Field.of("productSize", StandardSQLTypeName.FLOAT64), + Field.of("productUsage", StandardSQLTypeName.STRING)); kafkaResourceManager = KafkaResourceManager.builder(testName).setHost(TestProperties.hostIp()).build(); @@ -98,6 +102,7 @@ public void setup() throws IOException { URL avroSchemaResource = Resources.getResource("KafkaToBigQueryFlexAvroIT/avro_schema.avsc"); gcsClient.uploadArtifact("avro_schema.avsc", avroSchemaResource.getPath()); avroSchema = new org.apache.avro.Schema.Parser().parse(avroSchemaResource.openStream()); + avroSchemaUsageEnum = avroSchema.getField("productUsage").schema(); URL otherAvroSchemaResource = Resources.getResource("KafkaToBigQueryFlexAvroIT/other_avro_schema.avsc"); @@ -150,6 +155,10 @@ public void testKafkaToBigQueryAvroWithSchemaRegistry() throws IOException, Rest b -> b.addParameter("messageFormat", "AVRO_CONFLUENT_WIRE_FORMAT") .addParameter("schemaFormat", "SCHEMA_REGISTRY") + // If this test fails, check if the below schema registry has + // correct schemas registered with the following IDs: + // - 5 (avro_schema.avsc) + // - 4 (other_avro_schema.avsc) .addParameter("schemaRegistryConnectionUrl", "http://10.128.0.60:8081") .addParameter("writeMode", "DYNAMIC_TABLE_NAMES") .addParameter("outputProject", PROJECT) @@ -166,8 +175,10 @@ public void testKafkaToBigQueryAvroWithSchemaRegistryWithKey() b -> b.addParameter("messageFormat", "AVRO_CONFLUENT_WIRE_FORMAT") .addParameter("schemaFormat", "SCHEMA_REGISTRY") - // Schemas are registered with ids 3 and 4. If this test fails, check if the - // below schema registry address contains the expected schema registered. + // If this test fails, check if the below schema registry has + // correct schemas registered with the following IDs: + // - 5 (avro_schema.avsc) + // - 4 (other_avro_schema.avsc) .addParameter("schemaRegistryConnectionUrl", "http://10.128.0.60:8081") .addParameter("writeMode", "DYNAMIC_TABLE_NAMES") .addParameter("outputProject", PROJECT) @@ -399,10 +410,8 @@ private void baseKafkaToBigQueryAvro( && options.getParameter("schemaFormat").equals("SCHEMA_REGISTRY") && options.getParameter("schemaRegistryConnectionUrl") != null) { - // Schemas are registered in schema registry with IDs 3 and 4 for Kafka Reads. So for these - // tests - // publish the messages with schema IDs 3 and 4. - publishDoubleSchemaMessages(topicName, 3, 4); + // Schemas are registered with ids 5 (avro_schema.avsc) and 4 (other_avro_schema.avsc). + publishDoubleSchemaMessages(topicName, 5, 4); tableId = TableId.of(bqDatasetId, avroSchema.getFullName().replace(".", "-")); TableId otherTableId = TableId.of(bqDatasetId, otherAvroSchema.getFullName().replace(".", "-")); @@ -448,6 +457,10 @@ private void baseKafkaToBigQueryAvro( 11, "productName", "Dataflow", + "productSize", + 2.5d, + "productUsage", + "HIGH", "_key", Base64.getEncoder().encodeToString("11".getBytes())), Map.of( @@ -455,14 +468,34 @@ private void baseKafkaToBigQueryAvro( 12, "productName", "Pub/Sub", + "productSize", + 123.125d, + "productUsage", + "MEDIUM", "_key", Base64.getEncoder().encodeToString("12".getBytes())))); } else { assertThatBigQueryRecords(tableRows) .hasRecordsUnordered( List.of( - Map.of("productId", 11, "productName", "Dataflow"), - Map.of("productId", 12, "productName", "Pub/Sub"))); + Map.of( + "productId", + 11, + "productName", + "Dataflow", + "productSize", + 2.5d, + "productUsage", + "HIGH"), + Map.of( + "productId", + 12, + "productName", + "Pub/Sub", + "productSize", + 123.125d, + "productUsage", + "MEDIUM"))); } } @@ -476,13 +509,14 @@ private void publishSingleSchemaMessages(String topicName) new StringSerializer(), new KafkaAvroSerializer(registryClient)); for (int i = 1; i <= 10; i++) { - GenericRecord dataflow = createRecord(Integer.valueOf(i + "1"), "Dataflow", 0); + GenericRecord dataflow = createRecord(Integer.valueOf(i + "1"), "Dataflow", 2.5f, "HIGH"); publish(kafkaProducer, topicName, i + "1", dataflow); - GenericRecord pubsub = createRecord(Integer.valueOf(i + "2"), "Pub/Sub", 0); + GenericRecord pubsub = createRecord(Integer.valueOf(i + "2"), "Pub/Sub", 123.125f, "MEDIUM"); publish(kafkaProducer, topicName, i + "2", pubsub); - GenericRecord invalid = createRecord(Integer.valueOf(i + "3"), "InvalidNameTooLong", 0); + GenericRecord invalid = + createRecord(Integer.valueOf(i + "3"), "InvalidNameTooLong", 0f, "UNDEFINED"); publish(kafkaProducer, topicName, i + "3", invalid); try { @@ -504,26 +538,25 @@ private void publishDoubleSchemaMessages(String topicName, int schemaId1, int sc new StringSerializer(), new KafkaAvroSerializer(registryClient)); for (int i = 1; i <= 10; i++) { - GenericRecord dataflow = createRecord(Integer.valueOf(i + "1"), "Dataflow", 0); + GenericRecord dataflow = createRecord(Integer.valueOf(i + "1"), "Dataflow", 2.5f, "HIGH"); publish(kafkaProducer, topicName, i + "1", dataflow); - GenericRecord pubsub = createRecord(Integer.valueOf(i + "2"), "Pub/Sub", 0); + GenericRecord pubsub = createRecord(Integer.valueOf(i + "2"), "Pub/Sub", 123.125f, "MEDIUM"); publish(kafkaProducer, topicName, i + "2", pubsub); - GenericRecord invalid = createRecord(Integer.valueOf(i + "3"), "InvalidNameTooLong", 0); + GenericRecord invalid = + createRecord(Integer.valueOf(i + "3"), "InvalidNameTooLong", 0f, "UNDEFINED"); publish(kafkaProducer, topicName, i + "3", invalid); GenericRecord otherDataflow = - createOtherRecord(Integer.valueOf(i + "4"), "Dataflow", "dataflow", 0); + createOtherRecord(Integer.valueOf(i + "4"), "Dataflow", "dataflow"); publish(kafkaProducer, topicName, i + "4", otherDataflow); - GenericRecord otherPubsub = - createOtherRecord(Integer.valueOf(i + "5"), "Pub/Sub", "pubsub", 0); + GenericRecord otherPubsub = createOtherRecord(Integer.valueOf(i + "5"), "Pub/Sub", "pubsub"); publish(kafkaProducer, topicName, i + "5", otherPubsub); GenericRecord otherInvalid = - createOtherRecord( - Integer.valueOf(i + "6"), "InvalidNameTooLong", "InvalidNameTooLong", 0); + createOtherRecord(Integer.valueOf(i + "6"), "InvalidNameTooLong", "InvalidNameTooLong"); publish(kafkaProducer, topicName, i + "6", otherInvalid); try { @@ -540,13 +573,14 @@ public void publishBinaryMessages(String topicName) throws IOException { new StringSerializer(), new BinaryAvroSerializer(avroSchema)); for (int i = 1; i <= 10; i++) { - GenericRecord dataflow = createRecord(Integer.valueOf(i + "1"), "Dataflow", 0); + GenericRecord dataflow = createRecord(Integer.valueOf(i + "1"), "Dataflow", 2.5f, "HIGH"); publishBinary(kafkaProducer, topicName, i + "1", dataflow); - GenericRecord pubsub = createRecord(Integer.valueOf(i + "2"), "Pub/Sub", 0); + GenericRecord pubsub = createRecord(Integer.valueOf(i + "2"), "Pub/Sub", 123.125f, "MEDIUM"); publishBinary(kafkaProducer, topicName, i + "2", pubsub); - GenericRecord invalid = createRecord(Integer.valueOf(i + "3"), "InvalidNameTooLong", 0); + GenericRecord invalid = + createRecord(Integer.valueOf(i + "3"), "InvalidNameTooLong", 0f, "UNDEFINED"); publishBinary(kafkaProducer, topicName, i + "3", invalid); try { @@ -590,14 +624,17 @@ private void publishBinary( } } - private GenericRecord createRecord(int id, String productName, double value) { + private GenericRecord createRecord( + int id, String productName, float productSize, String productUsage) { return new GenericRecordBuilder(avroSchema) .set("productId", id) .set("productName", productName) + .set("productSize", productSize) + .set("productUsage", new GenericData.EnumSymbol(avroSchemaUsageEnum, productUsage)) .build(); } - private GenericRecord createOtherRecord(int id, String productName, String name, double value) { + private GenericRecord createOtherRecord(int id, String productName, String name) { return new GenericRecordBuilder(otherAvroSchema) .set("productId", id) .set("productName", productName) diff --git a/v2/kafka-to-bigquery/src/test/resources/KafkaToBigQueryFlexAvroIT/avro_schema.avsc b/v2/kafka-to-bigquery/src/test/resources/KafkaToBigQueryFlexAvroIT/avro_schema.avsc index d907a7f17a..a7e0fac42b 100644 --- a/v2/kafka-to-bigquery/src/test/resources/KafkaToBigQueryFlexAvroIT/avro_schema.avsc +++ b/v2/kafka-to-bigquery/src/test/resources/KafkaToBigQueryFlexAvroIT/avro_schema.avsc @@ -10,6 +10,24 @@ { "name": "productName", "type": "string" + }, + { + "name": "productSize", + "type": "float" + }, + { + "name": "productUsage", + "type": { + "type": "enum", + "name": "Usage", + "symbols": [ + "UNDEFINED", + "LOW", + "MEDIUM", + "HIGH" + ] + }, + "default": "UNDEFINED" } ] } \ No newline at end of file From a77d886b41573b8cbfa6988c094956052cbf734c Mon Sep 17 00:00:00 2001 From: darshan-sj Date: Tue, 3 Dec 2024 12:38:09 +0530 Subject: [PATCH 09/14] Parameterizing spanner Host in spanner tests (#2020) * Parameterizing spanner Host parameter in spanner tests * move testName cleanup logic to templateTestBase * spotless * Creating spannerTemplateITBase class for parameters for all SpannerITs * Adding comments * Adding a debug log * Spotless fix --- cicd/cmd/run-it-smoke-tests/main.go | 1 - cicd/cmd/run-it-tests/main.go | 1 - .../apache/beam/it/gcp/TemplateTestBase.java | 10 ++- .../gcp/spanner/SpannerResourceManager.java | 20 +++-- .../it/gcp/spanner/SpannerTemplateITBase.java | 42 ++++++++++ .../teleport/spanner/ExportPipelineIT.java | 30 ++----- .../teleport/spanner/ImportPipelineIT.java | 30 ++----- .../teleport/templates/SpannerToTextIT.java | 30 ++----- .../v2/templates/DataStreamToSpannerIT.java | 78 ++----------------- .../SpannerChangeStreamsToGcsIT.java | 26 ++----- 10 files changed, 90 insertions(+), 178 deletions(-) create mode 100644 it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerTemplateITBase.java diff --git a/cicd/cmd/run-it-smoke-tests/main.go b/cicd/cmd/run-it-smoke-tests/main.go index 5a2fc17fc1..e5fa462235 100644 --- a/cicd/cmd/run-it-smoke-tests/main.go +++ b/cicd/cmd/run-it-smoke-tests/main.go @@ -66,7 +66,6 @@ func main() { flags.ArtifactBucket(), flags.StageBucket(), flags.PrivateConnectivity(), - flags.SpannerHost(), flags.FailureMode(), flags.RetryFailures(), flags.StaticOracleHost(), diff --git a/cicd/cmd/run-it-tests/main.go b/cicd/cmd/run-it-tests/main.go index 8830edfb9b..24b8145d63 100644 --- a/cicd/cmd/run-it-tests/main.go +++ b/cicd/cmd/run-it-tests/main.go @@ -67,7 +67,6 @@ func main() { flags.StageBucket(), flags.HostIp(), flags.PrivateConnectivity(), - flags.SpannerHost(), flags.FailureMode(), flags.RetryFailures(), flags.StaticOracleHost(), diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/TemplateTestBase.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/TemplateTestBase.java index e3f70e4562..47923348de 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/TemplateTestBase.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/TemplateTestBase.java @@ -90,11 +90,13 @@ public abstract class TemplateTestBase { public TestRule watcher = new TestWatcher() { protected void starting(Description description) { - LOG.info( - "Starting integration test {}.{}", - description.getClassName(), - description.getMethodName()); testName = description.getMethodName(); + // In case of parameterization the testName can contain subscript like testName[paramName] + // Converting testName from testName[paramName] to testNameParamName since it is used to + // create many resources and it cannot contain special characters. + testName = testName.replaceAll("\\[", ""); + testName = testName.replaceAll("\\]", ""); + LOG.info("Starting integration test {}.{}", description.getClassName(), testName); } }; diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerResourceManager.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerResourceManager.java index 54ff99c5e6..73079eb00e 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerResourceManager.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerResourceManager.java @@ -81,7 +81,9 @@ public final class SpannerResourceManager implements ResourceManager { private static final Logger LOG = LoggerFactory.getLogger(SpannerResourceManager.class); private static final int MAX_BASE_ID_LENGTH = 30; - private static final String DEFAULT_SPANNER_HOST = "https://batch-spanner.googleapis.com"; + public static final String DEFAULT_SPANNER_HOST = "https://batch-spanner.googleapis.com"; + public static final String STAGING_SPANNER_HOST = + "https://staging-wrenchworks.sandbox.googleapis.com"; // Retry settings for instance creation private static final int CREATE_MAX_RETRIES = 5; @@ -232,7 +234,11 @@ private synchronized void maybeCreateDatabase() { private static RetryPolicy retryOnQuotaException() { return RetryPolicy.builder() - .handleIf(exception -> ExceptionUtils.containsMessage(exception, "RESOURCE_EXHAUSTED")) + .handleIf( + exception -> { + LOG.warn("Error from spanner:", exception); + return ExceptionUtils.containsMessage(exception, "RESOURCE_EXHAUSTED"); + }) .withMaxRetries(CREATE_MAX_RETRIES) .withBackoff(CREATE_BACKOFF_DELAY, CREATE_BACKOFF_MAX_DELAY) .withJitter(CREATE_BACKOFF_JITTER) @@ -597,15 +603,13 @@ public Builder setInstanceId(String instanceId) { } /** - * Looks at the system properties if there's a Spanner host override, uses it for Spanner API - * calls. + * Overrides spanner host, uses it for Spanner API calls. * + * @param spannerHost spanner host URL * @return this builder with host set. */ - public Builder maybeUseCustomHost() { - if (System.getProperty("spannerHost") != null) { - this.host = System.getProperty("spannerHost"); - } + public Builder useCustomHost(String spannerHost) { + this.host = spannerHost; return this; } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerTemplateITBase.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerTemplateITBase.java new file mode 100644 index 0000000000..84c109a71f --- /dev/null +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/spanner/SpannerTemplateITBase.java @@ -0,0 +1,42 @@ +package org.apache.beam.it.gcp.spanner; + +import java.util.Arrays; +import java.util.Collection; +import org.apache.beam.it.gcp.TemplateTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +/** + * Base class for all Spanner template integration tests. This class parameterizes the spannerHost + * for all subclasses. If the "spannerHost" system property is not set, the value of spannerHost + * will be set to STAGING_SPANNER_HOST and DEFAULT_SPANNER_HOST (as defined in {@link + * SpannerResourceManager}). All tests in the base class will be run twice: once with spannerHost + * set to STAGING_SPANNER_HOST and once with spannerHost set to DEFAULT_SPANNER_HOST. Otherwise, If + * the "spannerHost" system property is set, its value will be used to set spannerHost. All + * subclasses must use SpannerResourceManager.useCustomHost() and pass the spannerHost parameter to + * it. + */ +@RunWith(Parameterized.class) +public abstract class SpannerTemplateITBase extends TemplateTestBase { + + @Parameterized.Parameter(0) + public String spannerHost; + + @Parameterized.Parameter(1) + public String spannerHostName; + + // Because of parameterization, the test names will have subscripts. For example: + // testSpannerToGCSAvroBase[Staging] + @Parameters(name = "{1}") + public static Collection parameters() { + if (System.getProperty("spannerHost") != null) { + return Arrays.asList(new Object[][] {{System.getProperty("spannerHost"), "Custom"}}); + } + return Arrays.asList( + new Object[][] { + {SpannerResourceManager.STAGING_SPANNER_HOST, "Staging"}, + {SpannerResourceManager.DEFAULT_SPANNER_HOST, "Default"} + }); + } +} diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ExportPipelineIT.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ExportPipelineIT.java index 68003d4c26..bdeb29a5fd 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/ExportPipelineIT.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ExportPipelineIT.java @@ -34,23 +34,23 @@ import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.PipelineOperator; import org.apache.beam.it.common.utils.ResourceManagerUtils; -import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.artifacts.Artifact; import org.apache.beam.it.gcp.artifacts.utils.AvroTestUtil; import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerTemplateITBase; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.RandomUtils; import org.junit.After; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; /** Integration test for {@link ExportPipeline Spanner to GCS Avro} template. */ @Category(TemplateIntegrationTest.class) @TemplateIntegrationTest(ExportPipeline.class) -@RunWith(JUnit4.class) -public class ExportPipelineIT extends TemplateTestBase { +@RunWith(Parameterized.class) +public class ExportPipelineIT extends SpannerTemplateITBase { private static final int MESSAGES_COUNT = 100; @@ -154,16 +154,7 @@ public void testSpannerToGCSAvro() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.GOOGLE_STANDARD_SQL) .maybeUseStaticInstance() - .build(); - testSpannerToGCSAvroBase(Function.identity()); - } - - @Test - public void testSpannerToGCSAvroStaging() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.GOOGLE_STANDARD_SQL) - .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testSpannerToGCSAvroBase( paramAdder -> @@ -260,16 +251,7 @@ public void testPostgresSpannerToGCSAvro() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.POSTGRESQL) .maybeUseStaticInstance() - .build(); - testPostgresSpannerToGCSAvroBase(Function.identity()); - } - - @Test - public void testPostgresSpannerToGCSAvroStaging() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.POSTGRESQL) - .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testPostgresSpannerToGCSAvroBase( paramAdder -> diff --git a/v1/src/test/java/com/google/cloud/teleport/spanner/ImportPipelineIT.java b/v1/src/test/java/com/google/cloud/teleport/spanner/ImportPipelineIT.java index da8784de4b..87d45a2dcb 100644 --- a/v1/src/test/java/com/google/cloud/teleport/spanner/ImportPipelineIT.java +++ b/v1/src/test/java/com/google/cloud/teleport/spanner/ImportPipelineIT.java @@ -35,19 +35,19 @@ import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.PipelineOperator; import org.apache.beam.it.common.utils.ResourceManagerUtils; -import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerTemplateITBase; import org.junit.After; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; /** Integration test for {@link ImportPipeline} classic template. */ @Category(TemplateIntegrationTest.class) @TemplateIntegrationTest(ImportPipeline.class) -@RunWith(JUnit4.class) -public class ImportPipelineIT extends TemplateTestBase { +@RunWith(Parameterized.class) +public class ImportPipelineIT extends SpannerTemplateITBase { private SpannerResourceManager spannerResourceManager; @@ -127,16 +127,7 @@ public void testGoogleSqlImportPipeline() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.GOOGLE_STANDARD_SQL) .maybeUseStaticInstance() - .build(); - testGoogleSqlImportPipelineBase(Function.identity()); - } - - @Test - public void testGoogleSqlImportPipelineStaging() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.GOOGLE_STANDARD_SQL) - .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testGoogleSqlImportPipelineBase( paramAdder -> @@ -209,16 +200,7 @@ public void testPostgresImportPipeline() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.POSTGRESQL) .maybeUseStaticInstance() - .build(); - testPostgresImportPipelineBase(Function.identity()); - } - - @Test - public void testPostgresImportPipelineStaging() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.POSTGRESQL) - .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testPostgresImportPipelineBase( paramAdder -> diff --git a/v1/src/test/java/com/google/cloud/teleport/templates/SpannerToTextIT.java b/v1/src/test/java/com/google/cloud/teleport/templates/SpannerToTextIT.java index 41106cd578..5240495a16 100644 --- a/v1/src/test/java/com/google/cloud/teleport/templates/SpannerToTextIT.java +++ b/v1/src/test/java/com/google/cloud/teleport/templates/SpannerToTextIT.java @@ -32,21 +32,21 @@ import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.PipelineOperator; import org.apache.beam.it.common.utils.ResourceManagerUtils; -import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.artifacts.Artifact; import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerTemplateITBase; import org.apache.commons.lang3.RandomStringUtils; import org.junit.After; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; /** Integration test for {@link SpannerToText Spanner to GCS Text} template. */ @Category(TemplateIntegrationTest.class) @TemplateIntegrationTest(SpannerToText.class) -@RunWith(JUnit4.class) -public class SpannerToTextIT extends TemplateTestBase { +@RunWith(Parameterized.class) +public class SpannerToTextIT extends SpannerTemplateITBase { private static final int MESSAGES_COUNT = 100; @@ -62,16 +62,7 @@ public void testSpannerToGCSText() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.GOOGLE_STANDARD_SQL) .maybeUseStaticInstance() - .build(); - testSpannerToGCSTextBase(Function.identity()); - } - - @Test - public void testSpannerToGCSTextStaging() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.GOOGLE_STANDARD_SQL) - .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testSpannerToGCSTextBase( paramAdder -> @@ -145,16 +136,7 @@ public void testPostgresSpannerToGCSText() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.POSTGRESQL) .maybeUseStaticInstance() - .build(); - testPostgresSpannerToGCSTextBase(Function.identity()); - } - - @Test - public void testPostgresSpannerToGCSTextStaging() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION, Dialect.POSTGRESQL) - .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testPostgresSpannerToGCSTextBase( paramAdder -> diff --git a/v2/datastream-to-spanner/src/test/java/com/google/cloud/teleport/v2/templates/DataStreamToSpannerIT.java b/v2/datastream-to-spanner/src/test/java/com/google/cloud/teleport/v2/templates/DataStreamToSpannerIT.java index 7abd3fad40..22b8438a35 100644 --- a/v2/datastream-to-spanner/src/test/java/com/google/cloud/teleport/v2/templates/DataStreamToSpannerIT.java +++ b/v2/datastream-to-spanner/src/test/java/com/google/cloud/teleport/v2/templates/DataStreamToSpannerIT.java @@ -46,7 +46,6 @@ import org.apache.beam.it.common.utils.ResourceManagerUtils; import org.apache.beam.it.conditions.ChainedConditionCheck; import org.apache.beam.it.conditions.ConditionCheck; -import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.cloudsql.CloudMySQLResourceManager; import org.apache.beam.it.gcp.cloudsql.CloudOracleResourceManager; import org.apache.beam.it.gcp.cloudsql.CloudSqlResourceManager; @@ -56,6 +55,7 @@ import org.apache.beam.it.gcp.datastream.OracleSource; import org.apache.beam.it.gcp.pubsub.PubsubResourceManager; import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerTemplateITBase; import org.apache.beam.it.gcp.spanner.conditions.SpannerRowsCheck; import org.apache.beam.it.gcp.spanner.matchers.SpannerAsserts; import org.apache.beam.it.jdbc.JDBCResourceManager; @@ -65,13 +65,13 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; /** Integration test for {@link DataStreamToSpanner} Flex template. */ @Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) @TemplateIntegrationTest(DataStreamToSpanner.class) -@RunWith(JUnit4.class) -public class DataStreamToSpannerIT extends TemplateTestBase { +@RunWith(Parameterized.class) +public class DataStreamToSpannerIT extends SpannerTemplateITBase { enum JDBCType { MYSQL, @@ -126,7 +126,6 @@ public void testDataStreamMySqlToSpanner() throws IOException { simpleMySqlToSpannerTest( DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, Dialect.GOOGLE_STANDARD_SQL, - false, Function.identity()); } @@ -136,27 +135,6 @@ public void testDataStreamOracleToSpanner() throws IOException { simpleOracleToSpannerTest( DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, Dialect.GOOGLE_STANDARD_SQL, - false, - Function.identity()); - } - - @Test - public void testDataStreamMySqlToSpannerStaging() throws IOException { - // Run a simple IT - simpleMySqlToSpannerTest( - DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, - Dialect.GOOGLE_STANDARD_SQL, - true, - Function.identity()); - } - - @Test - public void testDataStreamOracleToSpannerStaging() throws IOException { - // Run a simple IT - simpleOracleToSpannerTest( - DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, - Dialect.GOOGLE_STANDARD_SQL, - true, Function.identity()); } @@ -166,17 +144,6 @@ public void testDataStreamMySqlToPostgresSpanner() throws IOException { simpleMySqlToSpannerTest( DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, Dialect.POSTGRESQL, - false, - Function.identity()); - } - - @Test - public void testDataStreamMySqlToPostgresSpannerStaging() throws IOException { - // Run a simple IT - simpleMySqlToSpannerTest( - DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, - Dialect.POSTGRESQL, - true, Function.identity()); } @@ -186,7 +153,6 @@ public void testDataStreamMySqlToSpannerStreamingEngine() throws IOException { simpleMySqlToSpannerTest( DatastreamResourceManager.DestinationOutputFormat.AVRO_FILE_FORMAT, Dialect.GOOGLE_STANDARD_SQL, - false, config -> config.addEnvironment("enableStreamingEngine", true)); } @@ -196,7 +162,6 @@ public void testDataStreamMySqlToSpannerJson() throws IOException { simpleMySqlToSpannerTest( DatastreamResourceManager.DestinationOutputFormat.JSON_FILE_FORMAT, Dialect.GOOGLE_STANDARD_SQL, - false, Function.identity()); } @@ -206,34 +171,12 @@ public void testDataStreamOracleToSpannerJson() throws IOException { simpleOracleToSpannerTest( DatastreamResourceManager.DestinationOutputFormat.JSON_FILE_FORMAT, Dialect.GOOGLE_STANDARD_SQL, - false, - Function.identity()); - } - - @Test - public void testDataStreamMySqlToSpannerJsonStaging() throws IOException { - // Run a simple IT - simpleMySqlToSpannerTest( - DatastreamResourceManager.DestinationOutputFormat.JSON_FILE_FORMAT, - Dialect.GOOGLE_STANDARD_SQL, - true, - Function.identity()); - } - - @Test - public void testDataStreamOracleToSpannerJsonStaging() throws IOException { - // Run a simple IT - simpleOracleToSpannerTest( - DatastreamResourceManager.DestinationOutputFormat.JSON_FILE_FORMAT, - Dialect.GOOGLE_STANDARD_SQL, - true, Function.identity()); } private void simpleMySqlToSpannerTest( DatastreamResourceManager.DestinationOutputFormat fileFormat, Dialect spannerDialect, - boolean isStaging, Function paramsAdder) throws IOException { @@ -241,7 +184,6 @@ private void simpleMySqlToSpannerTest( JDBCType.MYSQL, fileFormat, spannerDialect, - isStaging, config -> paramsAdder.apply( config.addParameter("sessionFilePath", getGcsPath("input/mysql-session.json")))); @@ -250,23 +192,17 @@ private void simpleMySqlToSpannerTest( private void simpleOracleToSpannerTest( DatastreamResourceManager.DestinationOutputFormat fileFormat, Dialect spannerDialect, - boolean isStaging, Function paramsAdder) throws IOException { simpleJdbcToSpannerTest( - JDBCType.ORACLE, - fileFormat, - spannerDialect, - isStaging, - config -> paramsAdder.apply(config)); + JDBCType.ORACLE, fileFormat, spannerDialect, config -> paramsAdder.apply(config)); } private void simpleJdbcToSpannerTest( JDBCType jdbcType, DatastreamResourceManager.DestinationOutputFormat fileFormat, Dialect spannerDialect, - boolean isStaging, Function paramsAdder) throws IOException { @@ -280,10 +216,8 @@ private void simpleJdbcToSpannerTest( SpannerResourceManager.Builder spannerResourceManagerBuilder = SpannerResourceManager.builder(testName, PROJECT, REGION, spannerDialect) .maybeUseStaticInstance() + .useCustomHost(spannerHost) .setCredentials(credentials); - if (isStaging) { - spannerResourceManagerBuilder.maybeUseCustomHost(); - } spannerResourceManager = spannerResourceManagerBuilder.build(); // Generate table names diff --git a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsIT.java b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsIT.java index 6fcd2dcbdb..6c92691155 100644 --- a/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsIT.java +++ b/v2/googlecloud-to-googlecloud/src/test/java/com/google/cloud/teleport/v2/templates/SpannerChangeStreamsToGcsIT.java @@ -38,11 +38,11 @@ import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.PipelineOperator; import org.apache.beam.it.common.utils.ResourceManagerUtils; -import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.artifacts.Artifact; import org.apache.beam.it.gcp.artifacts.utils.AvroTestUtil; import org.apache.beam.it.gcp.artifacts.utils.JsonTestUtil; import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerTemplateITBase; 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.commons.lang3.RandomStringUtils; @@ -50,15 +50,15 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; /** * Integration test for {@link SpannerChangeStreamsToGcs Spanner Change Streams to GCS} template. */ @Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) @TemplateIntegrationTest(SpannerChangeStreamsToGcs.class) -@RunWith(JUnit4.class) -public class SpannerChangeStreamsToGcsIT extends TemplateTestBase { +@RunWith(Parameterized.class) +public class SpannerChangeStreamsToGcsIT extends SpannerTemplateITBase { private static final int MESSAGES_COUNT = 20; private static final Pattern RESULT_REGEX = Pattern.compile(".*result-.*"); @@ -72,17 +72,10 @@ public void teardown() { @Test public void testSpannerChangeStreamsToGcs() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION).maybeUseStaticInstance().build(); - testSpannerChangeStreamsToGcsBase(Function.identity()); - } - - @Test - public void testSpannerChangeStreamsToGcsStaging() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION) .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testSpannerChangeStreamsToGcsBase( paramAdder -> @@ -190,17 +183,10 @@ private void testSpannerChangeStreamsToGcsBase( @Test public void testSpannerChangeStreamsToGcsAvro() throws IOException { - spannerResourceManager = - SpannerResourceManager.builder(testName, PROJECT, REGION).maybeUseStaticInstance().build(); - testSpannerChangeStreamsToGcsAvroBase(Function.identity()); - } - - @Test - public void testSpannerChangeStreamsToGcsAvroStaging() throws IOException { spannerResourceManager = SpannerResourceManager.builder(testName, PROJECT, REGION) .maybeUseStaticInstance() - .maybeUseCustomHost() + .useCustomHost(spannerHost) .build(); testSpannerChangeStreamsToGcsAvroBase( paramAdder -> From c37f7219002cd1b25fadcb68ef20973d06fc770e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 3 Dec 2024 10:29:04 -0500 Subject: [PATCH 10/14] Add extra validation to SpannerToBigQuery Template (#2035) * Add extra validation to SpannerToBigQuery Template * Formatting * Spotless --- .../cloud/teleport/v2/options/SpannerToBigQueryOptions.java | 2 +- .../cloud/teleport/v2/templates/SpannerToBigQuery.java | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerToBigQueryOptions.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerToBigQueryOptions.java index 1d6012b2e4..da1293d067 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerToBigQueryOptions.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/options/SpannerToBigQueryOptions.java @@ -96,7 +96,7 @@ public interface SpannerToBigQueryOptions optional = true, description = "Cloud Storage path to BigQuery JSON schema", helpText = - "The Cloud Storage path (gs://) to the JSON file that defines your BigQuery schema.", + "The Cloud Storage path (gs://) to the JSON file that defines your BigQuery schema. This is required if the Create Disposition is not CREATE_NEVER", example = "gs://your-bucket/your-schema.json") String getBigQuerySchemaPath(); diff --git a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToBigQuery.java b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToBigQuery.java index 2176e6f9ed..80be2f22d6 100644 --- a/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToBigQuery.java +++ b/v2/googlecloud-to-googlecloud/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToBigQuery.java @@ -79,6 +79,11 @@ public static void main(String[] args) { } else { throw new IllegalArgumentException("either sqlQuery or spannerTableId required"); } + if (Strings.isNullOrEmpty(options.getBigQuerySchemaPath()) + && CreateDisposition.valueOf(options.getCreateDisposition()) != CREATE_NEVER) { + throw new IllegalArgumentException( + "bigQuerySchemaPath is required if CreateDisposition is not CREATE_NEVER"); + } pipeline .apply(read) .apply(new StructToJson()) From ca317e08edada2be0d8892f69a96304bb60703d5 Mon Sep 17 00:00:00 2001 From: liferoad Date: Tue, 3 Dec 2024 11:01:06 -0500 Subject: [PATCH 11/14] Update add-flex-template.md (#2047) --- contributor-docs/add-flex-template.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contributor-docs/add-flex-template.md b/contributor-docs/add-flex-template.md index ab77d1f8e3..d1843e5881 100644 --- a/contributor-docs/add-flex-template.md +++ b/contributor-docs/add-flex-template.md @@ -348,11 +348,11 @@ such as missing JavaDocs. Once formatted, you can run (from the project's root as well): ```shell -mvn clean package -pl v2/wordcount -am -Dmaven.test.skip +mvn clean install -pl v2/wordcount -am -Dmaven.test.skip ``` The `-am` option guarantees that all the necessary local dependencies are -included in the build. +included in the build. You can ignore the error releated to `v2/wordcount` if any error occurs. `-pl v2/wordcount` is how we specify the target module, allowing us to only build what we need. You can see all the available modules in the From 09f68d23ef1cdf78cc0af8e51c2285d4fe1be615 Mon Sep 17 00:00:00 2001 From: shreyakhajanchi <92910380+shreyakhajanchi@users.noreply.github.com> Date: Wed, 4 Dec 2024 10:39:40 +0530 Subject: [PATCH 12/14] custom transformation implementation (#2040) * custom transformation implementation * adding a test * addressing comments --- v2/spanner-to-sourcedb/pom.xml | 6 + .../v2/templates/SpannerToSourceDb.java | 56 +++++- .../v2/templates/constants/Constants.java | 7 + .../dbutils/dml/MySQLDMLGenerator.java | 31 +++- .../processor/InputRecordProcessor.java | 45 ++++- .../templates/models/DMLGeneratorRequest.java | 15 ++ .../templates/transforms/SourceWriterFn.java | 43 ++++- .../transforms/SourceWriterTransform.java | 30 +++- .../dbutils/dml/MySQLDMLGeneratorTest.java | 27 +++ .../transforms/SourceWriterFnTest.java | 155 ++++++++++++++-- .../test/resources/customTransformation.json | 169 ++++++++++++++++++ 11 files changed, 548 insertions(+), 36 deletions(-) create mode 100644 v2/spanner-to-sourcedb/src/test/resources/customTransformation.json diff --git a/v2/spanner-to-sourcedb/pom.xml b/v2/spanner-to-sourcedb/pom.xml index 841405fb9a..fd7896b923 100644 --- a/v2/spanner-to-sourcedb/pom.xml +++ b/v2/spanner-to-sourcedb/pom.xml @@ -82,6 +82,12 @@ beam-it-jdbc test + + com.google.cloud.teleport.v2 + spanner-custom-shard + ${project.version} + test + com.google.cloud.teleport diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDb.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDb.java index e2cfa9a009..d33ec69b4e 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDb.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDb.java @@ -29,6 +29,7 @@ import com.google.cloud.teleport.v2.spanner.migrations.schema.Schema; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; import com.google.cloud.teleport.v2.spanner.migrations.spanner.SpannerSchema; +import com.google.cloud.teleport.v2.spanner.migrations.transformation.CustomTransformation; import com.google.cloud.teleport.v2.spanner.migrations.utils.SecretManagerAccessorImpl; import com.google.cloud.teleport.v2.spanner.migrations.utils.SessionFileReader; import com.google.cloud.teleport.v2.spanner.migrations.utils.ShardFileReader; @@ -367,6 +368,53 @@ public interface Options extends PipelineOptions, StreamingOptions { String getSourceType(); void setSourceType(String value); + + @TemplateParameter.GcsReadFile( + order = 25, + optional = true, + description = "Custom transformation jar location in Cloud Storage", + helpText = + "Custom jar location in Cloud Storage that contains the custom transformation logic for processing records" + + " in reverse replication.") + @Default.String("") + String getTransformationJarPath(); + + void setTransformationJarPath(String value); + + @TemplateParameter.Text( + order = 26, + optional = true, + description = "Custom class name for transformation", + helpText = + "Fully qualified class name having the custom transformation logic. It is a" + + " mandatory field in case transformationJarPath is specified") + @Default.String("") + String getTransformationClassName(); + + void setTransformationClassName(String value); + + @TemplateParameter.Text( + order = 27, + optional = true, + description = "Custom parameters for transformation", + helpText = + "String containing any custom parameters to be passed to the custom transformation class.") + @Default.String("") + String getTransformationCustomParameters(); + + void setTransformationCustomParameters(String value); + + @TemplateParameter.Text( + order = 28, + optional = true, + description = "Directory name for holding filtered records", + helpText = + "Records skipped from reverse replication are written to this directory. Default" + + " directory name is skip.") + @Default.String("filteredEvents") + String getFilterEventsDirectoryName(); + + void setFilterEventsDirectoryName(String value); } /** @@ -541,6 +589,11 @@ public static PipelineResult run(Options options) { } else { mergedRecords = dlqRecords; } + CustomTransformation customTransformation = + CustomTransformation.builder( + options.getTransformationJarPath(), options.getTransformationClassName()) + .setCustomParameters(options.getTransformationCustomParameters()) + .build(); SourceWriterTransform.Result sourceWriterOutput = mergedRecords .apply( @@ -578,7 +631,8 @@ public static PipelineResult run(Options options) { options.getShadowTablePrefix(), options.getSkipDirectoryName(), connectionPoolSizePerWorker, - options.getSourceType())); + options.getSourceType(), + customTransformation)); PCollection> dlqPermErrorRecords = reconsumedElements diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/constants/Constants.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/constants/Constants.java index 31ed2b8428..1368a46fe3 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/constants/Constants.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/constants/Constants.java @@ -59,6 +59,9 @@ public class Constants { // The Tag for skipped records public static final TupleTag SKIPPED_TAG = new TupleTag() {}; + // The Tag for records filtered via custom transformation. + public static final TupleTag FILTERED_TAG = new TupleTag() {}; + // Message written to the file for skipped records public static final String SKIPPED_TAG_MESSAGE = "Skipped record from reverse replication"; @@ -72,4 +75,8 @@ public class Constants { public static final String DEFAULT_SHARD_ID = "single_shard"; public static final String SOURCE_MYSQL = "mysql"; + + // Message written to the file for filtered records + public static final String FILTERED_TAG_MESSAGE = + "Filtered record from custom transformation in reverse replication"; } diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGenerator.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGenerator.java index 8774fedfa9..c06917bf87 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGenerator.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGenerator.java @@ -82,7 +82,8 @@ public DMLGeneratorResponse getDMLStatement(DMLGeneratorRequest dmlGeneratorRequ sourceTable, dmlGeneratorRequest.getNewValuesJson(), dmlGeneratorRequest.getKeyValuesJson(), - dmlGeneratorRequest.getSourceDbTimezoneOffset()); + dmlGeneratorRequest.getSourceDbTimezoneOffset(), + dmlGeneratorRequest.getCustomTransformationResponse()); if (pkcolumnNameValues == null) { LOG.warn( "Cannot reverse replicate for table {} without primary key, skipping the record", @@ -194,7 +195,8 @@ private static DMLGeneratorResponse generateUpsertStatement( sourceTable, dmlGeneratorRequest.getNewValuesJson(), dmlGeneratorRequest.getKeyValuesJson(), - dmlGeneratorRequest.getSourceDbTimezoneOffset()); + dmlGeneratorRequest.getSourceDbTimezoneOffset(), + dmlGeneratorRequest.getCustomTransformationResponse()); return getUpsertStatement( sourceTable.getName(), sourceTable.getPrimaryKeySet(), @@ -207,7 +209,8 @@ private static Map getColumnValues( SourceTable sourceTable, JSONObject newValuesJson, JSONObject keyValuesJson, - String sourceDbTimezoneOffset) { + String sourceDbTimezoneOffset, + Map customTransformationResponse) { Map response = new HashMap<>(); /* @@ -224,6 +227,10 @@ private static Map getColumnValues( as the column will be stored with default/null values */ Set sourcePKs = sourceTable.getPrimaryKeySet(); + Set customTransformColumns = null; + if (customTransformationResponse != null) { + customTransformColumns = customTransformationResponse.keySet(); + } for (Map.Entry entry : sourceTable.getColDefs().entrySet()) { SourceColumnDefinition sourceColDef = entry.getValue(); @@ -231,6 +238,10 @@ private static Map getColumnValues( if (sourcePKs.contains(colName)) { continue; // we only need non-primary keys } + if (customTransformColumns != null && customTransformColumns.contains(colName)) { + response.put(colName, customTransformationResponse.get(colName).toString()); + continue; + } String colId = entry.getKey(); SpannerColumnDefinition spannerColDef = spannerTable.getColDefs().get(colId); @@ -272,7 +283,8 @@ private static Map getPkColumnValues( SourceTable sourceTable, JSONObject newValuesJson, JSONObject keyValuesJson, - String sourceDbTimezoneOffset) { + String sourceDbTimezoneOffset, + Map customTransformationResponse) { Map response = new HashMap<>(); /* Get all primary key col ids from source table @@ -286,6 +298,10 @@ private static Map getPkColumnValues( if the column does not exist in any of the JSON - return null */ ColumnPK[] sourcePKs = sourceTable.getPrimaryKeys(); + Set customTransformColumns = null; + if (customTransformationResponse != null) { + customTransformColumns = customTransformationResponse.keySet(); + } for (int i = 0; i < sourcePKs.length; i++) { ColumnPK currentSourcePK = sourcePKs[i]; @@ -298,6 +314,13 @@ private static Map getPkColumnValues( sourceColDef.getName()); return null; } + if (customTransformColumns != null + && customTransformColumns.contains(sourceColDef.getName())) { + response.put( + sourceColDef.getName(), + customTransformationResponse.get(sourceColDef.getName()).toString()); + continue; + } String spannerColumnName = spannerColDef.getName(); String columnValue = ""; if (keyValuesJson.has(spannerColumnName)) { diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/processor/InputRecordProcessor.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/processor/InputRecordProcessor.java index 6fba8c3fe2..9bdfe2bcda 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/processor/InputRecordProcessor.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/processor/InputRecordProcessor.java @@ -15,7 +15,12 @@ */ package com.google.cloud.teleport.v2.templates.dbutils.processor; +import com.google.cloud.teleport.v2.spanner.exceptions.InvalidTransformationException; +import com.google.cloud.teleport.v2.spanner.migrations.convertors.ChangeEventToMapConvertor; import com.google.cloud.teleport.v2.spanner.migrations.schema.Schema; +import com.google.cloud.teleport.v2.spanner.utils.ISpannerMigrationTransformer; +import com.google.cloud.teleport.v2.spanner.utils.MigrationTransformationRequest; +import com.google.cloud.teleport.v2.spanner.utils.MigrationTransformationResponse; import com.google.cloud.teleport.v2.templates.changestream.TrimmedShardedDataChangeRecord; import com.google.cloud.teleport.v2.templates.dbutils.dao.source.IDao; import com.google.cloud.teleport.v2.templates.dbutils.dml.IDMLGenerator; @@ -23,10 +28,12 @@ import com.google.cloud.teleport.v2.templates.models.DMLGeneratorResponse; import java.time.Instant; import java.time.temporal.ChronoUnit; +import java.util.Map; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Metrics; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.joda.time.Duration; import org.json.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,14 +42,18 @@ public class InputRecordProcessor { private static final Logger LOG = LoggerFactory.getLogger(InputRecordProcessor.class); + private static final Distribution applyCustomTransformationResponseTimeMetric = + Metrics.distribution( + InputRecordProcessor.class, "apply_custom_transformation_impl_latency_ms"); - public static void processRecord( + public static boolean processRecord( TrimmedShardedDataChangeRecord spannerRecord, Schema schema, IDao dao, String shardId, String sourceDbTimezoneOffset, - IDMLGenerator dmlGenerator) + IDMLGenerator dmlGenerator, + ISpannerMigrationTransformer spannerToSourceTransformer) throws Exception { try { @@ -53,17 +64,43 @@ public static void processRecord( String newValueJsonStr = spannerRecord.getMod().getNewValuesJson(); JSONObject newValuesJson = new JSONObject(newValueJsonStr); JSONObject keysJson = new JSONObject(keysJsonStr); + Map customTransformationResponse = null; + if (spannerToSourceTransformer != null) { + org.joda.time.Instant startTimestamp = org.joda.time.Instant.now(); + Map mapRequest = + ChangeEventToMapConvertor.combineJsonObjects(keysJson, newValuesJson); + MigrationTransformationRequest migrationTransformationRequest = + new MigrationTransformationRequest(tableName, mapRequest, shardId, modType); + MigrationTransformationResponse migrationTransformationResponse = null; + try { + migrationTransformationResponse = + spannerToSourceTransformer.toSourceRow(migrationTransformationRequest); + } catch (Exception e) { + throw new InvalidTransformationException(e); + } + org.joda.time.Instant endTimestamp = org.joda.time.Instant.now(); + applyCustomTransformationResponseTimeMetric.update( + new Duration(startTimestamp, endTimestamp).getMillis()); + if (migrationTransformationResponse.isEventFiltered()) { + Metrics.counter(InputRecordProcessor.class, "filtered_events_" + shardId).inc(); + return true; + } + if (migrationTransformationResponse != null) { + customTransformationResponse = migrationTransformationResponse.getResponseRow(); + } + } DMLGeneratorRequest dmlGeneratorRequest = new DMLGeneratorRequest.Builder( modType, tableName, newValuesJson, keysJson, sourceDbTimezoneOffset) .setSchema(schema) + .setCustomTransformationResponse(customTransformationResponse) .build(); DMLGeneratorResponse dmlGeneratorResponse = dmlGenerator.getDMLStatement(dmlGeneratorRequest); if (dmlGeneratorResponse.getDmlStatement().isEmpty()) { LOG.warn("DML statement is empty for table: " + tableName); - return; + return false; } dao.write(dmlGeneratorResponse.getDmlStatement()); @@ -79,7 +116,7 @@ public static void processRecord( long replicationLag = ChronoUnit.SECONDS.between(commitTsInst, instTime); lagMetric.update(replicationLag); // update the lag metric - + return false; } catch (Exception e) { LOG.error( "The exception while processing shardId: {} is {} ", diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/models/DMLGeneratorRequest.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/models/DMLGeneratorRequest.java index 661f05038d..3db153c51e 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/models/DMLGeneratorRequest.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/models/DMLGeneratorRequest.java @@ -16,6 +16,7 @@ package com.google.cloud.teleport.v2.templates.models; import com.google.cloud.teleport.v2.spanner.migrations.schema.Schema; +import java.util.Map; import org.json.JSONObject; /** @@ -51,6 +52,8 @@ public class DMLGeneratorRequest { // The timezone offset of the source database, used for handling timezone-specific data. private final String sourceDbTimezoneOffset; + private Map customTransformationResponse; + public DMLGeneratorRequest(Builder builder) { this.modType = builder.modType; this.spannerTableName = builder.spannerTableName; @@ -58,6 +61,7 @@ public DMLGeneratorRequest(Builder builder) { this.newValuesJson = builder.newValuesJson; this.keyValuesJson = builder.keyValuesJson; this.sourceDbTimezoneOffset = builder.sourceDbTimezoneOffset; + this.customTransformationResponse = builder.customTransformationResponse; } public String getModType() { @@ -84,6 +88,10 @@ public String getSourceDbTimezoneOffset() { return sourceDbTimezoneOffset; } + public Map getCustomTransformationResponse() { + return customTransformationResponse; + } + public static class Builder { private final String modType; private final String spannerTableName; @@ -91,6 +99,7 @@ public static class Builder { private final JSONObject keyValuesJson; private final String sourceDbTimezoneOffset; private Schema schema; + private Map customTransformationResponse; public Builder( String modType, @@ -110,6 +119,12 @@ public Builder setSchema(Schema schema) { return this; } + public Builder setCustomTransformationResponse( + Map customTransformationResponse) { + this.customTransformationResponse = customTransformationResponse; + return this; + } + public DMLGeneratorRequest build() { return new DMLGeneratorRequest(this); } diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFn.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFn.java index f4af6d9780..6b00511bf8 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFn.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFn.java @@ -23,10 +23,14 @@ import com.google.cloud.teleport.v2.spanner.ddl.Ddl; import com.google.cloud.teleport.v2.spanner.ddl.IndexColumn; import com.google.cloud.teleport.v2.spanner.ddl.Table; +import com.google.cloud.teleport.v2.spanner.exceptions.InvalidTransformationException; import com.google.cloud.teleport.v2.spanner.migrations.convertors.ChangeEventSpannerConvertor; import com.google.cloud.teleport.v2.spanner.migrations.exceptions.ChangeEventConvertorException; import com.google.cloud.teleport.v2.spanner.migrations.schema.Schema; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; +import com.google.cloud.teleport.v2.spanner.migrations.transformation.CustomTransformation; +import com.google.cloud.teleport.v2.spanner.migrations.utils.CustomTransformationImplFetcher; +import com.google.cloud.teleport.v2.spanner.utils.ISpannerMigrationTransformer; import com.google.cloud.teleport.v2.templates.changestream.ChangeStreamErrorRecord; import com.google.cloud.teleport.v2.templates.changestream.TrimmedShardedDataChangeRecord; import com.google.cloud.teleport.v2.templates.constants.Constants; @@ -75,6 +79,9 @@ public class SourceWriterFn extends DoFn shards; @@ -86,6 +93,8 @@ public class SourceWriterFn extends DoFn shards, @@ -96,7 +105,8 @@ public SourceWriterFn( String shadowTablePrefix, String skipDirName, int maxThreadPerDataflowWorker, - String source) { + String source, + CustomTransformation customTransformation) { this.schema = schema; this.sourceDbTimezoneOffset = sourceDbTimezoneOffset; @@ -107,6 +117,7 @@ public SourceWriterFn( this.skipDirName = skipDirName; this.maxThreadPerDataflowWorker = maxThreadPerDataflowWorker; this.source = source; + this.customTransformation = customTransformation; } // for unit testing purposes @@ -124,6 +135,12 @@ public void setSourceProcessor(SourceProcessor sourceProcessor) { this.sourceProcessor = sourceProcessor; } + // for unit testing purposes + public void setSpannerToSourceTransformer( + ISpannerMigrationTransformer spannerToSourceTransformer) { + this.spannerToSourceTransformer = spannerToSourceTransformer; + } + /** Setup function connects to Cloud Spanner. */ @Setup public void setup() throws UnsupportedSourceException { @@ -132,6 +149,8 @@ public void setup() throws UnsupportedSourceException { sourceProcessor = SourceProcessorFactory.createSourceProcessor(source, shards, maxThreadPerDataflowWorker); spannerDao = new SpannerDao(spannerConfig); + spannerToSourceTransformer = + CustomTransformationImplFetcher.getCustomTransformationLogicImpl(customTransformation); } /** Teardown function disconnects from the Cloud Spanner. */ @@ -184,13 +203,18 @@ public void processElement(ProcessContext c) { if (!isSourceAhead) { IDao sourceDao = sourceProcessor.getSourceDao(shardId); - InputRecordProcessor.processRecord( - spannerRec, - schema, - sourceDao, - shardId, - sourceDbTimezoneOffset, - sourceProcessor.getDmlGenerator()); + boolean isEventFiltered = + InputRecordProcessor.processRecord( + spannerRec, + schema, + sourceDao, + shardId, + sourceDbTimezoneOffset, + sourceProcessor.getDmlGenerator(), + spannerToSourceTransformer); + if (isEventFiltered) { + outputWithTag(c, Constants.FILTERED_TAG, Constants.FILTERED_TAG_MESSAGE, spannerRec); + } spannerDao.updateShadowTable( getShadowTableMutation( @@ -206,6 +230,9 @@ public void processElement(ProcessContext c) { } com.google.cloud.Timestamp timestamp = com.google.cloud.Timestamp.now(); c.output(Constants.SUCCESS_TAG, timestamp.toString()); + } catch (InvalidTransformationException ex) { + invalidTransformationException.inc(); + outputWithTag(c, Constants.PERMANENT_ERROR_TAG, ex.getMessage(), spannerRec); } catch (ChangeEventConvertorException ex) { outputWithTag(c, Constants.PERMANENT_ERROR_TAG, ex.getMessage(), spannerRec); } catch (SpannerException diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterTransform.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterTransform.java index e80f617eeb..ef9ddbfaac 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterTransform.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterTransform.java @@ -19,6 +19,7 @@ import com.google.cloud.teleport.v2.spanner.ddl.Ddl; import com.google.cloud.teleport.v2.spanner.migrations.schema.Schema; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; +import com.google.cloud.teleport.v2.spanner.migrations.transformation.CustomTransformation; import com.google.cloud.teleport.v2.templates.changestream.TrimmedShardedDataChangeRecord; import com.google.cloud.teleport.v2.templates.constants.Constants; import com.google.common.base.Preconditions; @@ -52,6 +53,7 @@ public class SourceWriterTransform private final String skipDirName; private final int maxThreadPerDataflowWorker; private final String source; + private final CustomTransformation customTransformation; public SourceWriterTransform( List shards, @@ -62,7 +64,8 @@ public SourceWriterTransform( String shadowTablePrefix, String skipDirName, int maxThreadPerDataflowWorker, - String source) { + String source, + CustomTransformation customTransformation) { this.schema = schema; this.sourceDbTimezoneOffset = sourceDbTimezoneOffset; @@ -73,6 +76,7 @@ public SourceWriterTransform( this.skipDirName = skipDirName; this.maxThreadPerDataflowWorker = maxThreadPerDataflowWorker; this.source = source; + this.customTransformation = customTransformation; } @Override @@ -91,18 +95,21 @@ public SourceWriterTransform.Result expand( this.shadowTablePrefix, this.skipDirName, this.maxThreadPerDataflowWorker, - this.source)) + this.source, + this.customTransformation)) .withOutputTags( Constants.SUCCESS_TAG, TupleTagList.of(Constants.PERMANENT_ERROR_TAG) .and(Constants.RETRYABLE_ERROR_TAG) - .and(Constants.SKIPPED_TAG))); + .and(Constants.SKIPPED_TAG) + .and(Constants.FILTERED_TAG))); return Result.create( sourceWriteResults.get(Constants.SUCCESS_TAG), sourceWriteResults.get(Constants.PERMANENT_ERROR_TAG), sourceWriteResults.get(Constants.RETRYABLE_ERROR_TAG), - sourceWriteResults.get(Constants.SKIPPED_TAG)); + sourceWriteResults.get(Constants.SKIPPED_TAG), + sourceWriteResults.get(Constants.FILTERED_TAG)); } /** Container class for the results of this transform. */ @@ -113,13 +120,18 @@ private static Result create( PCollection successfulSourceWrites, PCollection permanentErrors, PCollection retryableErrors, - PCollection skippedSourceWrites) { + PCollection skippedSourceWrites, + PCollection filteredWrites) { Preconditions.checkNotNull(successfulSourceWrites); Preconditions.checkNotNull(permanentErrors); Preconditions.checkNotNull(retryableErrors); Preconditions.checkNotNull(skippedSourceWrites); return new AutoValue_SourceWriterTransform_Result( - successfulSourceWrites, permanentErrors, retryableErrors, skippedSourceWrites); + successfulSourceWrites, + permanentErrors, + retryableErrors, + skippedSourceWrites, + filteredWrites); } public abstract PCollection successfulSourceWrites(); @@ -130,6 +142,8 @@ private static Result create( public abstract PCollection skippedSourceWrites(); + public abstract PCollection filteredWrites(); + @Override public void finishSpecifyingOutput( String transformName, PInput input, PTransform transform) { @@ -151,7 +165,9 @@ public Map, PValue> expand() { Constants.RETRYABLE_ERROR_TAG, retryableErrors(), Constants.SKIPPED_TAG, - skippedSourceWrites()); + skippedSourceWrites(), + Constants.FILTERED_TAG, + filteredWrites()); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGeneratorTest.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGeneratorTest.java index b01d139a6d..2d110e80ae 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGeneratorTest.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/MySQLDMLGeneratorTest.java @@ -1088,6 +1088,33 @@ public void testSpannerColDefsNull() { assertTrue(sql.isEmpty()); } + @Test + public void customTransformationMatch() { + Schema schema = SessionFileReader.read("src/test/resources/customTransformation.json"); + String tableName = "Singers"; + String newValuesString = "{\"FirstName\":\"kk\",\"LastName\":\"ll\"}"; + JSONObject newValuesJson = new JSONObject(newValuesString); + String keyValueString = "{\"SingerId\":\"999\"}"; + JSONObject keyValuesJson = new JSONObject(keyValueString); + String modType = "INSERT"; + Map customTransformation = new HashMap<>(); + customTransformation.put("FullName", "\'kk ll\'"); + customTransformation.put("SingerId", "1"); + + MySQLDMLGenerator mySQLDMLGenerator = new MySQLDMLGenerator(); + DMLGeneratorResponse dmlGeneratorResponse = + mySQLDMLGenerator.getDMLStatement( + new DMLGeneratorRequest.Builder( + modType, tableName, newValuesJson, keyValuesJson, "+00:00") + .setSchema(schema) + .setCustomTransformationResponse(customTransformation) + .build()); + String sql = dmlGeneratorResponse.getDmlStatement(); + + assertTrue(sql.contains("`FullName` = 'kk ll'")); + assertTrue(sql.contains("VALUES (1,'kk ll')")); + } + public static Schema getSchemaObject() { Map syntheticPKeys = new HashMap(); Map srcSchema = new HashMap(); diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFnTest.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFnTest.java index f2bac7247d..039429b0ce 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFnTest.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/transforms/SourceWriterFnTest.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates.transforms; +import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.contains; import static org.mockito.ArgumentMatchers.eq; @@ -29,6 +30,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.Timestamp; import com.google.cloud.teleport.v2.spanner.ddl.Ddl; +import com.google.cloud.teleport.v2.spanner.exceptions.InvalidTransformationException; import com.google.cloud.teleport.v2.spanner.migrations.schema.ColumnPK; import com.google.cloud.teleport.v2.spanner.migrations.schema.NameAndCols; import com.google.cloud.teleport.v2.spanner.migrations.schema.Schema; @@ -38,7 +40,10 @@ import com.google.cloud.teleport.v2.spanner.migrations.schema.SpannerTable; import com.google.cloud.teleport.v2.spanner.migrations.schema.SyntheticPKey; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; +import com.google.cloud.teleport.v2.spanner.migrations.transformation.CustomTransformation; import com.google.cloud.teleport.v2.spanner.migrations.utils.SessionFileReader; +import com.google.cloud.teleport.v2.spanner.utils.ISpannerMigrationTransformer; +import com.google.cloud.teleport.v2.spanner.utils.MigrationTransformationResponse; import com.google.cloud.teleport.v2.templates.changestream.ChangeStreamErrorRecord; import com.google.cloud.teleport.v2.templates.changestream.TrimmedShardedDataChangeRecord; import com.google.cloud.teleport.v2.templates.constants.Constants; @@ -63,6 +68,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.runners.MethodSorters; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; @@ -76,6 +82,7 @@ public class SourceWriterFnTest { @Mock HashMap mockDaoMap; @Mock private SpannerConfig mockSpannerConfig; @Mock private DoFn.ProcessContext processContext; + @Mock private ISpannerMigrationTransformer mockSpannerMigrationTransformer; private static Gson gson = new Gson(); private Shard testShard; @@ -148,7 +155,8 @@ public void testSourceIsAhead() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -175,7 +183,8 @@ public void testSourceIsAheadWithSameCommitTimestamp() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -201,7 +210,8 @@ public void testSourceIsBehind() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -213,6 +223,118 @@ public void testSourceIsBehind() throws Exception { verify(mockSpannerDao, atLeast(1)).updateShadowTable(any()); } + @Test + public void testCustomTransformationException() throws Exception { + TrimmedShardedDataChangeRecord record = getParent1TrimmedDataChangeRecord("shardA"); + record.setShard("shardA"); + when(processContext.element()).thenReturn(KV.of(1L, record)); + when(mockSpannerMigrationTransformer.toSourceRow(any())) + .thenThrow(new InvalidTransformationException("some exception")); + CustomTransformation customTransformation = + CustomTransformation.builder("jarPath", "classPath").build(); + SourceWriterFn sourceWriterFn = + new SourceWriterFn( + ImmutableList.of(testShard), + testSchema, + mockSpannerConfig, + testSourceDbTimezoneOffset, + testDdl, + "shadow_", + "skip", + 500, + "mysql", + customTransformation); + ObjectMapper mapper = new ObjectMapper(); + mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + sourceWriterFn.setObjectMapper(mapper); + sourceWriterFn.setSourceProcessor(sourceProcessor); + sourceWriterFn.setSpannerDao(mockSpannerDao); + sourceWriterFn.setSpannerToSourceTransformer(mockSpannerMigrationTransformer); + sourceWriterFn.processElement(processContext); + verify(mockSpannerDao, atLeast(1)).getShadowTableRecord(any(), any()); + String jsonRec = gson.toJson(record, TrimmedShardedDataChangeRecord.class); + ChangeStreamErrorRecord errorRecord = + new ChangeStreamErrorRecord( + jsonRec, + "com.google.cloud.teleport.v2.spanner.exceptions.InvalidTransformationException: some exception"); + verify(processContext, atLeast(1)) + .output( + Constants.PERMANENT_ERROR_TAG, gson.toJson(errorRecord, ChangeStreamErrorRecord.class)); + } + + @Test + public void testCustomTransformationApplied() throws Exception { + TrimmedShardedDataChangeRecord record = getParent1TrimmedDataChangeRecord("shardA"); + record.setShard("shardA"); + when(processContext.element()).thenReturn(KV.of(1L, record)); + when(mockSpannerMigrationTransformer.toSourceRow(any())) + .thenReturn(new MigrationTransformationResponse(Map.of("id", "45"), false)); + CustomTransformation customTransformation = + CustomTransformation.builder("jarPath", "classPath").build(); + SourceWriterFn sourceWriterFn = + new SourceWriterFn( + ImmutableList.of(testShard), + testSchema, + mockSpannerConfig, + testSourceDbTimezoneOffset, + testDdl, + "shadow_", + "skip", + 500, + "mysql", + customTransformation); + ObjectMapper mapper = new ObjectMapper(); + mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + sourceWriterFn.setObjectMapper(mapper); + sourceWriterFn.setSourceProcessor(sourceProcessor); + sourceWriterFn.setSpannerDao(mockSpannerDao); + sourceWriterFn.setSpannerToSourceTransformer(mockSpannerMigrationTransformer); + sourceWriterFn.processElement(processContext); + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); + verify(mockSpannerDao, atLeast(1)).getShadowTableRecord(any(), any()); + verify(mockSqlDao, atLeast(1)).write(argumentCaptor.capture()); + assertTrue(argumentCaptor.getValue().contains("INSERT INTO `parent1`(`id`) VALUES (45)")); + verify(mockSpannerDao, atLeast(1)).updateShadowTable(any()); + } + + @Test + public void testCustomTransformationFiltered() throws Exception { + TrimmedShardedDataChangeRecord record = getParent1TrimmedDataChangeRecord("shardA"); + record.setShard("shardA"); + when(processContext.element()).thenReturn(KV.of(1L, record)); + when(mockSpannerMigrationTransformer.toSourceRow(any())) + .thenReturn(new MigrationTransformationResponse(null, true)); + CustomTransformation customTransformation = + CustomTransformation.builder("jarPath", "classPath").build(); + SourceWriterFn sourceWriterFn = + new SourceWriterFn( + ImmutableList.of(testShard), + testSchema, + mockSpannerConfig, + testSourceDbTimezoneOffset, + testDdl, + "shadow_", + "skip", + 500, + "mysql", + customTransformation); + ObjectMapper mapper = new ObjectMapper(); + mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); + sourceWriterFn.setObjectMapper(mapper); + sourceWriterFn.setSourceProcessor(sourceProcessor); + sourceWriterFn.setSpannerDao(mockSpannerDao); + sourceWriterFn.setSpannerToSourceTransformer(mockSpannerMigrationTransformer); + sourceWriterFn.processElement(processContext); + verify(mockSpannerDao, atLeast(1)).getShadowTableRecord(any(), any()); + verify(mockSqlDao, atLeast(0)).write(any()); + verify(mockSpannerDao, atLeast(0)).updateShadowTable(any()); + String jsonRec = gson.toJson(record, TrimmedShardedDataChangeRecord.class); + ChangeStreamErrorRecord errorRecord = + new ChangeStreamErrorRecord(jsonRec, Constants.FILTERED_TAG_MESSAGE); + verify(processContext, atLeast(1)) + .output(Constants.FILTERED_TAG, gson.toJson(errorRecord, ChangeStreamErrorRecord.class)); + } + @Test public void testNoShard() throws Exception { TrimmedShardedDataChangeRecord record = getParent1TrimmedDataChangeRecord("shardA"); @@ -227,7 +349,8 @@ public void testNoShard() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -257,7 +380,8 @@ public void testSkipShard() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -285,7 +409,8 @@ public void testPermanentError() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -317,7 +442,8 @@ public void testRetryableError() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -345,7 +471,8 @@ public void testRetryableErrorForForeignKey() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -375,7 +502,8 @@ public void testRetryableErrorConnectionFailure() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -405,7 +533,8 @@ public void testPermanentConnectionFailure() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -435,7 +564,8 @@ public void testPermanentGenericException() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); @@ -464,7 +594,8 @@ public void testDMLEmpty() throws Exception { "shadow_", "skip", 500, - "mysql"); + "mysql", + null); ObjectMapper mapper = new ObjectMapper(); mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS); sourceWriterFn.setObjectMapper(mapper); diff --git a/v2/spanner-to-sourcedb/src/test/resources/customTransformation.json b/v2/spanner-to-sourcedb/src/test/resources/customTransformation.json new file mode 100644 index 0000000000..320d184e7c --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/customTransformation.json @@ -0,0 +1,169 @@ +{ + "SessionName": "NewSession", + "EditorName": "", + "DatabaseType": "mysql", + "DatabaseName": "ui_demo", + "Dialect": "google_standard_sql", + "Notes": null, + "Tags": null, + "SpSchema": { + "t1": { + "Name": "Singers", + "ColIds": [ + "c5", + "c6", + "c7" + ], + "ColDefs": { + "c5": { + "Name": "SingerId", + "T": { + "Name": "INT64", + "Len": 0, + "IsArray": false + }, + "NotNull": true, + "Comment": "From: SingerId int", + "Id": "c5" + }, + "c6": { + "Name": "FirstName", + "T": { + "Name": "STRING", + "Len": 1024, + "IsArray": false + }, + "NotNull": false, + "Comment": "From: FirstName varchar(1024)", + "Id": "c6" + }, + "c7": { + "Name": "LastName", + "T": { + "Name": "STRING", + "Len": 1024, + "IsArray": false + }, + "NotNull": false, + "Comment": "From: LastName varchar(1024)", + "Id": "c7" + } + }, + "PrimaryKeys": [ + { + "ColId": "c5", + "Desc": false, + "Order": 1 + } + ], + "ForeignKeys": null, + "Indexes": [ + { + "Name": "ind1", + "TableId": "t1", + "Unique": false, + "Keys": [ + { + "ColId": "c5", + "Desc": false, + "Order": 1 + } + ], + "Id": "i9", + "StoredColumnIds": null + } + ], + "ParentId": "", + "Comment": "Spanner schema for source table Singers", + "Id": "t1" + } + }, + "SyntheticPKeys": {}, + "SrcSchema": { + "t1": { + "Name": "Singers", + "Schema": "ui_demo", + "ColIds": [ + "c5", + "c6" + ], + "ColDefs": { + "c5": { + "Name": "SingerId", + "Type": { + "Name": "int", + "Mods": null, + "ArrayBounds": null + }, + "NotNull": true, + "Ignored": { + "Check": false, + "Identity": false, + "Default": false, + "Exclusion": false, + "ForeignKey": false, + "AutoIncrement": false + }, + "Id": "c5" + }, + "c6": { + "Name": "FullName", + "Type": { + "Name": "varchar", + "Mods": [ + 1024 + ], + "ArrayBounds": null + }, + "NotNull": false, + "Ignored": { + "Check": false, + "Identity": false, + "Default": false, + "Exclusion": false, + "ForeignKey": false, + "AutoIncrement": false + }, + "Id": "c6" + } + }, + "PrimaryKeys": [ + { + "ColId": "c5", + "Desc": false, + "Order": 1 + } + ], + "ForeignKeys": null, + "Indexes": [ + { + "Name": "ind1", + "Unique": false, + "Keys": [ + { + "ColId": "c5", + "Desc": false, + "Order": 1 + } + ], + "Id": "i9", + "StoredColumnIds": null + } + ], + "Id": "t1" + } + }, + "SchemaIssues": { + "t1": { + "c5": [ + 13, + 18 + ] + } + }, + "Location": {}, + "TimezoneOffset": "+00:00", + "SpDialect": "google_standard_sql", + "UniquePKey": {}, + "Rules": [] +} \ No newline at end of file From ff14f97efbc47f2a13a9bd91f76e04b43967a1e9 Mon Sep 17 00:00:00 2001 From: shreyakhajanchi <92910380+shreyakhajanchi@users.noreply.github.com> Date: Thu, 5 Dec 2024 17:33:06 +0530 Subject: [PATCH 13/14] Adding dataflow timestamp field in datastream-common json flow (#2042) * Adding dataflow timestamp field in datastream-common json flow * updated UT --- .../FormatDatastreamJsonToJson.java | 5 +++ .../FormatDatastreamJsonToJsonTest.java | 37 ++++++++++++++++++- 2 files changed, 40 insertions(+), 2 deletions(-) diff --git a/v2/datastream-common/src/main/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJson.java b/v2/datastream-common/src/main/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJson.java index bd15edcba4..15de08e0bf 100644 --- a/v2/datastream-common/src/main/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJson.java +++ b/v2/datastream-common/src/main/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJson.java @@ -76,6 +76,7 @@ record = new ObjectMapper().readTree(c.element()); outputObject.put("_metadata_stream", getStreamName(record)); outputObject.put("_metadata_timestamp", getSourceTimestamp(record)); outputObject.put("_metadata_read_timestamp", getMetadataTimestamp(record)); + outputObject.put("_metadata_dataflow_timestamp", getCurrentTimestamp()); outputObject.put("_metadata_read_method", record.get("read_method").textValue()); outputObject.put("_metadata_source_type", sourceType); @@ -234,4 +235,8 @@ private Boolean getMetadataIsDeleted(JsonNode record) { return value.booleanValue(); } + + private long getCurrentTimestamp() { + return System.currentTimeMillis() / 1000L; + } } diff --git a/v2/datastream-common/src/test/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJsonTest.java b/v2/datastream-common/src/test/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJsonTest.java index 3794820df9..5faaeb55ca 100644 --- a/v2/datastream-common/src/test/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJsonTest.java +++ b/v2/datastream-common/src/test/java/com/google/cloud/teleport/v2/datastream/transforms/FormatDatastreamJsonToJsonTest.java @@ -15,6 +15,10 @@ */ package com.google.cloud.teleport.v2.datastream.transforms; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.cloud.teleport.v2.coders.FailsafeElementCoder; import com.google.cloud.teleport.v2.values.FailsafeElement; import com.google.common.collect.ImmutableMap; @@ -23,6 +27,7 @@ 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.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.junit.Rule; @@ -59,6 +64,9 @@ public void testProcessElement_validJson() { FailsafeElement expectedElement = FailsafeElement.of(EXAMPLE_DATASTREAM_RECORD, EXAMPLE_DATASTREAM_RECORD); + FailsafeElementCoder failsafeElementCoder = + FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + PCollection> pCollection = pipeline .apply("CreateInput", Create.of(EXAMPLE_DATASTREAM_JSON)) @@ -70,7 +78,9 @@ public void testProcessElement_validJson() { .withStreamName("my-stream") .withRenameColumnValues(renameColumns) .withLowercaseSourceColumns(false))) - .setCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); + .setCoder(failsafeElementCoder) + .apply("RemoveTimestampProperty", ParDo.of(new RemoveTimestampPropertyFn())) + .setCoder(failsafeElementCoder); PAssert.that(pCollection).containsInAnyOrder(expectedElement); @@ -85,6 +95,9 @@ public void testProcessElement_hashRowId() { FailsafeElement.of( EXAMPLE_DATASTREAM_RECORD_WITH_HASH_ROWID, EXAMPLE_DATASTREAM_RECORD_WITH_HASH_ROWID); + FailsafeElementCoder failsafeElementCoder = + FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()); + PCollection> pCollection = pipeline .apply("CreateInput", Create.of(EXAMPLE_DATASTREAM_JSON)) @@ -97,10 +110,30 @@ public void testProcessElement_hashRowId() { .withRenameColumnValues(renameColumns) .withHashRowId(true) .withLowercaseSourceColumns(false))) - .setCoder(FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); + .setCoder(failsafeElementCoder) + .apply("RemoveDataflowTimestampProperty", ParDo.of(new RemoveTimestampPropertyFn())) + .setCoder(failsafeElementCoder); PAssert.that(pCollection).containsInAnyOrder(expectedElement); pipeline.run(); } + + // Static nested DoFn class to remove timestamp property + static class RemoveTimestampPropertyFn + extends DoFn, FailsafeElement> { + + @ProcessElement + public void processElement( + @Element FailsafeElement element, + OutputReceiver> out) + throws JsonProcessingException { + ObjectMapper mapper = new ObjectMapper(); + JsonNode changeEvent = mapper.readTree(element.getPayload()); + if (changeEvent instanceof ObjectNode) { + ((ObjectNode) changeEvent).remove("_metadata_dataflow_timestamp"); + } + out.output(FailsafeElement.of(changeEvent.toString(), changeEvent.toString())); + } + } } From 91b5be3b344de3fe62f53776df259498f8b5d21e Mon Sep 17 00:00:00 2001 From: Vardhan Vinay Thigle <39047439+VardhanThigle@users.noreply.github.com> Date: Fri, 6 Dec 2024 09:55:32 +0000 Subject: [PATCH 14/14] minor typo in logs (#2046) --- .../cloud/teleport/v2/options/OptionsToConfigBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java index 9f75fc9f0c..e17e1eef50 100644 --- a/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java +++ b/v2/sourcedb-to-spanner/src/main/java/com/google/cloud/teleport/v2/options/OptionsToConfigBuilder.java @@ -172,7 +172,7 @@ protected static String mysqlSetCursorModeIfNeeded( SQLDialect sqlDialect, String url, @Nullable Integer fetchSize) { if (fetchSize == null) { LOG.info( - "FetchSize is not explicitly configured. In case of out of memory errors, please set `FetSize` according to the available memory and maximum size of a row."); + "FetchSize is not explicitly configured. In case of out of memory errors, please set `FetchSize` according to the available memory and maximum size of a row."); return url; } if (sqlDialect != SQLDialect.MYSQL) {