From db1f62722986ab03ae2a1522c7dc4004d9cd6772 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Tue, 28 Jan 2025 18:56:09 +0530 Subject: [PATCH 01/19] Add IT for Cassandra (#86) * Add IT for Cassandra Reverse Replication --- v2/spanner-to-sourcedb/pom.xml | 11 + .../dbutils/dml/CassandraTypeHandler.java | 17 +- .../CassandraSharedResourceManager.java | 373 ++++++++++++++++++ .../templates/SpannerToCassandraDbITBase.java | 239 +++++++++++ .../SpannerToCassandraSourceDbDatatypeIT.java | 316 +++++++++++++++ .../SpannerToSourceDbCassandraIT.java | 223 +++++++++++ .../dbutils/dml/CassandraTypeHandlerTest.java | 34 ++ .../cassandra-config-template.conf | 12 + .../cassandra-schema.sql | 43 ++ .../spanner-schema.sql | 49 +++ .../cassandra-config-template.conf | 12 + .../cassandra-schema.sql | 5 + .../spanner-schema.sql | 16 + 13 files changed, 1348 insertions(+), 2 deletions(-) create mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java create mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java create mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java create mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java create mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf create mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql create mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql create mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-config-template.conf create mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql create mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql diff --git a/v2/spanner-to-sourcedb/pom.xml b/v2/spanner-to-sourcedb/pom.xml index 634998da7c..6e790693cb 100644 --- a/v2/spanner-to-sourcedb/pom.xml +++ b/v2/spanner-to-sourcedb/pom.xml @@ -106,6 +106,17 @@ ${project.version} test + + org.apache.beam + beam-it-cassandra + test + + + com.datastax.cassandra + cassandra-driver-core + + + diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java index 2b3b183a5d..82a10d02de 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java @@ -121,6 +121,11 @@ private interface HandlerSupplier { private static BigInteger handleCassandraVarintType(Object value) { if (value instanceof byte[]) { return new BigInteger((byte[]) value); + } else if (value instanceof ByteBuffer) { + ByteBuffer byteBuffer = (ByteBuffer) value; + byte[] byteArray = new byte[byteBuffer.remaining()]; + byteBuffer.get(byteArray); + return new BigInteger(byteArray); } return new BigInteger(value.toString()); } @@ -172,10 +177,11 @@ private static ByteBuffer parseBlobType(Object colValue) { byteArray = (byte[]) colValue; } else if (colValue instanceof String) { byteArray = java.util.Base64.getDecoder().decode((String) colValue); + } else if (colValue instanceof ByteBuffer) { + return (ByteBuffer) colValue; } else { throw new IllegalArgumentException("Unsupported type for column"); } - return ByteBuffer.wrap(byteArray); } @@ -331,7 +337,14 @@ private static Object handleSpannerColumnType( return null; } String hexEncodedString = valuesJson.optString(columnName); - return convertBinaryEncodedStringToByteArray(hexEncodedString); + return safeHandle( + () -> { + try { + return convertBinaryEncodedStringToByteArray(hexEncodedString); + } catch (IllegalArgumentException e) { + return parseBlobType(hexEncodedString); + } + }); } else { return valuesJson.isNull(columnName) ? null : valuesJson.opt(columnName); } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java new file mode 100644 index 0000000000..f60ff147ef --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java @@ -0,0 +1,373 @@ +/* + * Copyright (C) 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package com.google.cloud.teleport.v2.templates; + +import com.datastax.oss.driver.api.core.CqlSession; +import com.datastax.oss.driver.api.core.DriverTimeoutException; +import com.datastax.oss.driver.api.core.cql.ResultSet; +import com.datastax.oss.driver.api.core.cql.Row; +import com.datastax.oss.driver.api.core.cql.SimpleStatement; +import dev.failsafe.Failsafe; +import dev.failsafe.RetryPolicy; +import java.net.InetSocketAddress; +import java.time.Duration; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import org.apache.beam.it.common.ResourceManager; +import org.apache.beam.it.common.utils.ExceptionUtils; +import org.apache.beam.it.common.utils.ResourceManagerUtils; +import org.apache.beam.it.testcontainers.TestContainerResourceManager; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.CassandraContainer; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.utility.DockerImageName; + +/** + * Client for managing Cassandra resources. + * + *

The class supports one database and multiple collections per database object. A database is + * created when the first collection is created if one has not been created already. + * + *

The database name is formed using testId. The database name will be "{testId}-{ISO8601 time, + * microsecond precision}", with additional formatting. + * + *

The class is thread-safe. + */ +public class CassandraSharedResourceManager + extends TestContainerResourceManager> implements ResourceManager { + + private static final Logger LOG = LoggerFactory.getLogger(CassandraSharedResourceManager.class); + + private static final String DEFAULT_CASSANDRA_CONTAINER_NAME = "cassandra"; + + // A list of available Cassandra Docker image tags can be found at + // https://hub.docker.com/_/cassandra/tags + private static final String DEFAULT_CASSANDRA_CONTAINER_TAG = "4.1.0"; + + // 9042 is the default port that Cassandra is configured to listen on + private static final int CASSANDRA_INTERNAL_PORT = 9042; + + private final CqlSession cassandraClient; + private final String keyspaceName; + private final boolean usingStaticDatabase; + + private CassandraSharedResourceManager(Builder builder) { + this( + /* cassandraClient= */ null, + new CassandraContainer<>( + DockerImageName.parse(builder.containerImageName).withTag(builder.containerImageTag)), + builder); + } + + @VisibleForTesting + @SuppressWarnings("nullness") + CassandraSharedResourceManager( + @Nullable CqlSession cassandraClient, CassandraContainer container, Builder builder) { + super(container, builder); + // we are trying to handle userDefined KeyspaceName name without usingStatic Container + this.usingStaticDatabase = builder.keyspaceName != null && !builder.preGeneratedKeyspaceName; + this.keyspaceName = + usingStaticDatabase || builder.preGeneratedKeyspaceName + ? builder.keyspaceName + : generateKeyspaceName(builder.testId); + this.cassandraClient = + cassandraClient == null + ? CqlSession.builder() + .addContactPoint( + new InetSocketAddress(this.getHost(), this.getPort(CASSANDRA_INTERNAL_PORT))) + .withLocalDatacenter("datacenter1") + .build() + : cassandraClient; + + if (!usingStaticDatabase) { + // Keyspace request may timeout on a few environments, if Cassandra is warming up + Failsafe.with(buildRetryPolicy()) + .run( + () -> + this.cassandraClient.execute( + String.format( + "CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}", + this.keyspaceName))); + } + } + + private String generateKeyspaceName(String testName) { + return ResourceManagerUtils.generateResourceId( + testName, + Pattern.compile("[/\\\\. \"\u0000$]"), + "-", + 27, + DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss-SSSSSS")) + .replace('-', '_'); + } + + public static Builder builder(String testId) { + return new Builder(testId); + } + + /** Returns the port to connect to the Cassandra Database. */ + public int getPort() { + return super.getPort(CASSANDRA_INTERNAL_PORT); + } + + /** + * Returns the name of the Database that this Cassandra manager will operate in. + * + * @return the name of the Cassandra Database. + */ + public synchronized String getKeyspaceName() { + return keyspaceName; + } + + /** + * Execute the given statement on the managed keyspace. + * + * @param statement The statement to execute. + * @return ResultSet from Cassandra. + */ + public synchronized ResultSet executeStatement(String statement) { + LOG.info("Executing statement: {}", statement); + + try { + return Failsafe.with(buildRetryPolicy()) + .get( + () -> + cassandraClient.execute( + SimpleStatement.newInstance(statement).setKeyspace(this.keyspaceName))); + } catch (Exception e) { + System.out.println(e.getMessage()); + System.out.println(e.fillInStackTrace()); + System.out.println(e); + throw new IllegalArgumentException("Error reading collection.", e); + } + } + + /** + * Execute the given statement on the managed keyspace without returning ResultSet. + * + * @param statement The statement to execute. + */ + public synchronized void execute(String statement) { + LOG.info("execute statement: {}", statement); + + try { + Failsafe.with(buildRetryPolicy()) + .run( + () -> + cassandraClient.execute( + SimpleStatement.newInstance(statement).setKeyspace(this.keyspaceName))); + } catch (Exception e) { + System.out.println(e.getMessage()); + System.out.println(e.fillInStackTrace()); + System.out.println(e); + throw new IllegalArgumentException("Error reading collection.", e); + } + } + + /** + * Inserts the given Document into a table. + * + *

A database will be created here, if one does not already exist. + * + * @param tableName The name of the table to insert the document into. + * @param document The document to insert into the table. + * @return A boolean indicating whether the Document was inserted successfully. + */ + public synchronized boolean insertDocument(String tableName, Map document) { + return insertDocuments(tableName, ImmutableList.of(document)); + } + + /** + * Inserts the given Documents into a collection. + * + *

Note: Implementations may do collection creation here, if one does not already exist. + * + * @param tableName The name of the collection to insert the documents into. + * @param documents A list of documents to insert into the collection. + * @return A boolean indicating whether the Documents were inserted successfully. + * @throws IllegalArgumentException if there is an error inserting the documents. + */ + public synchronized boolean insertDocuments(String tableName, List> documents) + throws IllegalArgumentException { + LOG.info( + "Attempting to write {} documents to {}.{}.", documents.size(), keyspaceName, tableName); + + try { + for (Map document : documents) { + executeStatement(createInsertStatement(tableName, document)); + } + } catch (Exception e) { + throw new IllegalArgumentException("Error inserting documents.", e); + } + + LOG.info("Successfully wrote {} documents to {}.{}", documents.size(), keyspaceName, tableName); + + return true; + } + + /** + * Reads all the Documents in a collection. + * + * @param tableName The name of the collection to read from. + * @return An iterable of all the Documents in the collection. + * @throws IllegalArgumentException if there is an error reading the collection. + */ + public synchronized Iterable readTable(String tableName) throws IllegalArgumentException { + LOG.info("Reading all documents from {}.{}", keyspaceName, tableName); + + Iterable documents; + try { + ResultSet resultSet = executeStatement(String.format("SELECT * FROM %s", tableName)); + documents = resultSet.all(); + } catch (Exception e) { + throw new IllegalArgumentException("Error reading table.", e); + } + + LOG.info("Successfully loaded documents from {}.{}", keyspaceName, tableName); + + return documents; + } + + @Override + public synchronized void cleanupAll() { + LOG.info("Attempting to cleanup Cassandra manager."); + + boolean producedError = false; + + // First, delete the database if it was not given as a static argument + if (!usingStaticDatabase) { + try { + executeStatement(String.format("DROP KEYSPACE IF EXISTS %s", this.keyspaceName)); + } catch (Exception e) { + LOG.error("Failed to drop Cassandra keyspace {}.", keyspaceName, e); + + // Only bubble exception if the cause is not timeout or does not exist + if (!ExceptionUtils.containsType(e, DriverTimeoutException.class) + && !ExceptionUtils.containsMessage(e, "does not exist")) { + producedError = true; + } + } + } + + // Next, try to close the Cassandra client connection + try { + cassandraClient.close(); + } catch (Exception e) { + LOG.error("Failed to delete Cassandra client.", e); + producedError = true; + } + + // Throw Exception at the end if there were any errors + if (producedError) { + throw new IllegalArgumentException("Failed to delete resources. Check above for errors."); + } + + super.cleanupAll(); + + LOG.info("Cassandra manager successfully cleaned up."); + } + + private String createInsertStatement(String tableName, Map map) { + StringBuilder columns = new StringBuilder(); + StringBuilder values = new StringBuilder(); + + for (Map.Entry entry : map.entrySet()) { + columns.append(entry.getKey()).append(", "); + + // add quotes around strings + if (entry.getValue() instanceof String) { + values.append("'").append(entry.getValue()).append("'"); + } else { + values.append(entry.getValue()); + } + values.append(", "); + } + + // Remove trailing comma and space + if (!map.isEmpty()) { + columns.delete(columns.length() - 2, columns.length()); + values.delete(values.length() - 2, values.length()); + } + + return String.format("INSERT INTO %s (%s) VALUES (%s)", tableName, columns, values); + } + + private static RetryPolicy buildRetryPolicy() { + return RetryPolicy.builder() + .withMaxRetries(5) + .withDelay(Duration.ofSeconds(1)) + .handle(DriverTimeoutException.class) + .build(); + } + + /** Builder for {@link CassandraSharedResourceManager}. */ + public static final class Builder + extends TestContainerResourceManager.Builder { + + private @Nullable String keyspaceName; + + private @Nullable boolean preGeneratedKeyspaceName; + + private Builder(String testId) { + super(testId, DEFAULT_CASSANDRA_CONTAINER_NAME, DEFAULT_CASSANDRA_CONTAINER_TAG); + this.keyspaceName = null; + } + + /** + * Sets the keyspace name to that of a preGeneratedKeyspaceName database instance. + * + *

Note: if a database name is set, and a static Cassandra server is being used + * (useStaticContainer() is also called on the builder), then a database will be created on the + * static server if it does not exist, and it will not be removed when cleanupAll() is called on + * the CassandraResourceManager. + * + * @param keyspaceName The database name. + * @return this builder object with the database name set. + */ + public Builder setKeyspaceName(String keyspaceName) { + this.keyspaceName = keyspaceName; + return this; + } + + /** + * Sets the preGeneratedKeyspaceName to that of a static database instance. Use this method only + * when attempting to operate on a pre-existing Cassandra database. + * + *

Note: if a database name is set, and a static Cassandra server is being used + * (useStaticContainer() is also called on the builder), then a database will be created on the + * static server if it does not exist, and it will not be removed when cleanupAll() is called on + * the CassandraResourceManager. + * + * @param preGeneratedKeyspaceName The database name. + * @return this builder object with the database is need to create. + */ + public Builder sePreGeneratedKeyspaceName(boolean preGeneratedKeyspaceName) { + this.preGeneratedKeyspaceName = preGeneratedKeyspaceName; + return this; + } + + @Override + public CassandraSharedResourceManager build() { + return new CassandraSharedResourceManager(this); + } + } +} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java new file mode 100644 index 0000000000..1d122c6894 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java @@ -0,0 +1,239 @@ +/* + * Copyright (C) 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package com.google.cloud.teleport.v2.templates; + +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; + +import com.google.cloud.teleport.v2.spanner.migrations.transformation.CustomTransformation; +import com.google.common.io.Resources; +import com.google.pubsub.v1.SubscriptionName; +import com.google.pubsub.v1.TopicName; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.regex.Pattern; +import org.apache.beam.it.common.PipelineLauncher; +import org.apache.beam.it.common.utils.PipelineUtils; +import org.apache.beam.it.common.utils.ResourceManagerUtils; +import org.apache.beam.it.gcp.TemplateTestBase; +import org.apache.beam.it.gcp.artifacts.utils.ArtifactUtils; +import org.apache.beam.it.gcp.pubsub.PubsubResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.storage.GcsResourceManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class SpannerToCassandraDbITBase extends TemplateTestBase { + private static final Logger LOG = LoggerFactory.getLogger(SpannerToCassandraDbITBase.class); + + protected SpannerResourceManager createSpannerDatabase(String spannerSchemaFile) + throws IOException { + SpannerResourceManager spannerResourceManager = + SpannerResourceManager.builder("rr-main-" + testName, PROJECT, REGION) + .maybeUseStaticInstance() + .build(); + String ddl = + String.join( + " ", + Resources.readLines(Resources.getResource(spannerSchemaFile), StandardCharsets.UTF_8)); + ddl = ddl.trim(); + String[] ddls = ddl.split(";"); + for (String d : ddls) { + if (!d.isBlank()) { + spannerResourceManager.executeDdlStatement(d); + } + } + return spannerResourceManager; + } + + protected SpannerResourceManager createSpannerMetadataDatabase() throws IOException { + SpannerResourceManager spannerMetadataResourceManager = + SpannerResourceManager.builder("rr-meta-" + testName, PROJECT, REGION) + .maybeUseStaticInstance() + .build(); + String dummy = "create table t1(id INT64 ) primary key(id)"; + spannerMetadataResourceManager.executeDdlStatement(dummy); + return spannerMetadataResourceManager; + } + + public PubsubResourceManager setUpPubSubResourceManager() throws IOException { + return PubsubResourceManager.builder(testName, PROJECT, credentialsProvider).build(); + } + + public CassandraSharedResourceManager generateKeyspaceAndBuildCassandraResource() { + String keyspaceName = + ResourceManagerUtils.generateResourceId( + testName, + Pattern.compile("[/\\\\. \"\u0000$]"), + "-", + 27, + DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss-SSSSSS")) + .replace('-', '_'); + if (keyspaceName.length() > 48) { + keyspaceName = keyspaceName.substring(0, 48); + } + return CassandraSharedResourceManager.builder(testName) + .setKeyspaceName(keyspaceName) + .sePreGeneratedKeyspaceName(true) + .build(); + } + + public SubscriptionName createPubsubResources( + String identifierSuffix, PubsubResourceManager pubsubResourceManager, String gcsPrefix) { + String topicNameSuffix = "rr-it" + identifierSuffix; + String subscriptionNameSuffix = "rr-it-sub" + identifierSuffix; + TopicName topic = pubsubResourceManager.createTopic(topicNameSuffix); + SubscriptionName subscription = + pubsubResourceManager.createSubscription(topic, subscriptionNameSuffix); + String prefix = gcsPrefix; + if (prefix.startsWith("/")) { + prefix = prefix.substring(1); + } + prefix += "/retry/"; + gcsClient.createNotification(topic.toString(), prefix); + return subscription; + } + + public void createAndUploadCassandraConfigToGcs( + GcsResourceManager gcsResourceManager, + CassandraSharedResourceManager cassandraResourceManagers) + throws IOException { + + String host = cassandraResourceManagers.getHost(); + int port = cassandraResourceManagers.getPort(); + String keyspaceName = cassandraResourceManagers.getKeyspaceName(); + System.out.println("Cassandra keyspaceName :: {}" + keyspaceName); + System.out.println("Cassandra host :: {}" + host); + System.out.println("Cassandra port :: {}" + port); + String cassandraConfigContents; + try (InputStream inputStream = + Thread.currentThread() + .getContextClassLoader() + .getResourceAsStream("SpannerToCassandraSourceIT/cassandra-config-template.conf")) { + if (inputStream == null) { + throw new FileNotFoundException( + "Resource file not found: SpannerToCassandraSourceIT/cassandra-config-template.conf"); + } + cassandraConfigContents = new String(inputStream.readAllBytes(), StandardCharsets.UTF_8); + } + + cassandraConfigContents = + cassandraConfigContents + .replace("##host##", host) + .replace("##port##", Integer.toString(port)) + .replace("##keyspace##", keyspaceName); + + System.out.println("Cassandra file contents: {}" + cassandraConfigContents); + gcsResourceManager.createArtifact("input/cassandra-config.conf", cassandraConfigContents); + } + + public PipelineLauncher.LaunchInfo launchDataflowJob( + GcsResourceManager gcsResourceManager, + SpannerResourceManager spannerResourceManager, + SpannerResourceManager spannerMetadataResourceManager, + String subscriptionName, + String identifierSuffix, + String shardingCustomJarPath, + String shardingCustomClassName, + String sourceDbTimezoneOffset, + CustomTransformation customTransformation) + throws IOException { + + Map params = + new HashMap<>() { + { + put("instanceId", spannerResourceManager.getInstanceId()); + put("databaseId", spannerResourceManager.getDatabaseId()); + put("spannerProjectId", PROJECT); + put("metadataDatabase", spannerMetadataResourceManager.getDatabaseId()); + put("metadataInstance", spannerMetadataResourceManager.getInstanceId()); + put( + "sourceShardsFilePath", + getGcsPath("input/cassandra-config.conf", gcsResourceManager)); + put("changeStreamName", "allstream"); + put("dlqGcsPubSubSubscription", subscriptionName); + put("deadLetterQueueDirectory", getGcsPath("dlq", gcsResourceManager)); + put("maxShardConnections", "5"); + put("maxNumWorkers", "1"); + put("numWorkers", "1"); + put("sourceType", "cassandra"); + } + }; + + if (shardingCustomJarPath != null) { + params.put( + "shardingCustomJarPath", + getGcsFullPath(gcsResourceManager, shardingCustomJarPath, identifierSuffix)); + } + if (shardingCustomClassName != null) { + params.put("shardingCustomClassName", shardingCustomClassName); + } + + if (sourceDbTimezoneOffset != null) { + params.put("sourceDbTimezoneOffset", sourceDbTimezoneOffset); + } + + if (customTransformation != null) { + params.put( + "transformationJarPath", getGcsPath(customTransformation.jarPath(), gcsResourceManager)); + params.put("transformationClassName", customTransformation.classPath()); + } + + // Construct template + String jobName = PipelineUtils.createJobName("rrev-it" + testName); + // /-DunifiedWorker=true when using runner v2 + PipelineLauncher.LaunchConfig.Builder options = + PipelineLauncher.LaunchConfig.builder(jobName, specPath); + options.setParameters(params); + options.addEnvironment("additionalExperiments", Collections.singletonList("use_runner_v2")); + // Run + PipelineLauncher.LaunchInfo jobInfo = launchTemplate(options, false); + assertThatPipeline(jobInfo).isRunning(); + return jobInfo; + } + + protected void createCassandraSchema( + CassandraSharedResourceManager cassandraResourceManager, String cassandraSchemaFile) + throws IOException { + String ddl = + String.join( + " ", + Resources.readLines( + Resources.getResource(cassandraSchemaFile), StandardCharsets.UTF_8)); + ddl = ddl.trim(); + System.out.println("DDL {}" + ddl); + String[] ddls = ddl.split(";"); + System.out.println("DDLs statement {}" + Arrays.toString(ddls)); + for (String d : ddls) { + System.out.println("DDL statement {}" + d); + if (!d.isBlank()) { + cassandraResourceManager.execute(d); + } + } + } + + public String getGcsFullPath( + GcsResourceManager gcsResourceManager, String artifactId, String identifierSuffix) { + return ArtifactUtils.getFullGcsPath( + artifactBucketName, identifierSuffix, gcsResourceManager.runId(), artifactId); + } +} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java new file mode 100644 index 0000000000..4634653bc5 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java @@ -0,0 +1,316 @@ +/* + * Copyright (C) 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package com.google.cloud.teleport.v2.templates; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; + +import com.datastax.oss.driver.api.core.cql.ResultSet; +import com.datastax.oss.driver.api.core.cql.Row; +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; +import com.google.cloud.teleport.metadata.TemplateIntegrationTest; +import com.google.pubsub.v1.SubscriptionName; +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +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.pubsub.PubsubResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.storage.GcsResourceManager; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.junit.runners.model.MultipleFailureException; + +@Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) +@TemplateIntegrationTest(SpannerToSourceDb.class) +@RunWith(JUnit4.class) +public class SpannerToCassandraSourceDbDatatypeIT extends SpannerToCassandraDbITBase { + + private static final String SPANNER_DDL_RESOURCE = + "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; + private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = + "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; + + private static final String TABLE = "AllDatatypeColumns"; + private static final HashSet testInstances = + new HashSet<>(); + private static PipelineLauncher.LaunchInfo jobInfo; + public static SpannerResourceManager spannerResourceManager; + private static SpannerResourceManager spannerMetadataResourceManager; + public static CassandraSharedResourceManager cassandraResourceManager; + private static GcsResourceManager gcsResourceManager; + private static PubsubResourceManager pubsubResourceManager; + private SubscriptionName subscriptionName; + private final List assertionErrors = new ArrayList<>(); + + /** + * Setup resource managers and Launch dataflow job once during the execution of this test class. + * + * @throws IOException + */ + @Before + public void setUp() throws IOException { + skipBaseCleanup = true; + synchronized (SpannerToCassandraSourceDbDatatypeIT.class) { + testInstances.add(this); + if (jobInfo == null) { + spannerResourceManager = createSpannerDatabase(SPANNER_DDL_RESOURCE); + spannerMetadataResourceManager = createSpannerMetadataDatabase(); + + cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); + gcsResourceManager = + GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) + .build(); + createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); + createAndUploadCassandraConfigToGcs(gcsResourceManager, cassandraResourceManager); + pubsubResourceManager = setUpPubSubResourceManager(); + subscriptionName = + createPubsubResources( + getClass().getSimpleName(), + pubsubResourceManager, + getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); + jobInfo = + launchDataflowJob( + gcsResourceManager, + spannerResourceManager, + spannerMetadataResourceManager, + subscriptionName.toString(), + null, + null, + null, + null, + null); + } + } + } + + /** + * Cleanup dataflow job and all the resources and resource managers. + * + * @throws IOException + */ + @AfterClass + public static void cleanUp() throws IOException { + for (SpannerToCassandraSourceDbDatatypeIT instance : testInstances) { + instance.tearDownBase(); + } + ResourceManagerUtils.cleanResources( + spannerResourceManager, + cassandraResourceManager, + spannerMetadataResourceManager, + gcsResourceManager, + pubsubResourceManager); + } + + @Test + public void spannerToCassandraSourceDataTypeConversionTest() + throws InterruptedException, IOException, MultipleFailureException { + assertThatPipeline(jobInfo).isRunning(); + writeRowInSpanner(); + assertRowInCassandraDB(); + } + + private long getRowCount() { + String query = String.format("SELECT COUNT(*) FROM %s", TABLE); + ResultSet resultSet = cassandraResourceManager.executeStatement(query); + Row row = resultSet.one(); + if (row != null) { + return row.getLong(0); + } else { + throw new RuntimeException("Query did not return a result for table: " + TABLE); + } + } + + private void writeRowInSpanner() { + Mutation mutation = + Mutation.newInsertOrUpdateBuilder(TABLE) + .set("varchar_column") + .to("SampleVarchar") + .set("tinyint_column") + .to(127) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(32767) + .set("mediumint_column") + .to(8388607) + .set("int_column") + .to(2147483647) + .set("bigint_column") + .to(9223372036854775807L) + .set("float_column") + .to(3.14159) + .set("double_column") + .to(2.718281828459045) + .set("decimal_column") + .to(new BigDecimal("12345.6789")) + .set("datetime_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(Value.bool(Boolean.TRUE)) + .set("other_bool_column") + .to(Value.bool(Boolean.FALSE)) + .set("bytes_column") + .to(Value.bytes(ByteArray.copyFrom("SGVsbG8gd29ybGQ=".getBytes()))) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to(Value.bytes(ByteArray.copyFrom("b3f5ed4f".getBytes()))) + .build(); + + spannerResourceManager.write(mutation); + } + + private void assertAll(Runnable... assertions) throws MultipleFailureException { + for (Runnable assertion : assertions) { + try { + assertion.run(); + } catch (AssertionError e) { + assertionErrors.add(e); + } + } + if (!assertionErrors.isEmpty()) { + throw new MultipleFailureException(assertionErrors); + } + } + + private void assertRowInCassandraDB() throws InterruptedException, MultipleFailureException { + PipelineOperator.Result result = + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); + assertThatResult(result).meetsConditions(); + Iterable rows; + try { + rows = cassandraResourceManager.readTable(TABLE); + } catch (Exception e) { + throw new RuntimeException("Failed to read from Cassandra table: " + TABLE, e); + } + + assertThat(rows).hasSize(1); + + Row row = rows.iterator().next(); + System.out.println(row.getFormattedContents()); + + assertThat(rows).hasSize(1); + assertAll( + () -> assertThat(row.getString("varchar_column")).isEqualTo("value1"), + () -> assertThat(row.getInt("tinyint_column")).isEqualTo(10), + () -> assertThat(row.getString("text_column")).isEqualTo("text_column_value"), + () -> + assertThat(row.getLocalDate("date_column")) + .isEqualTo(java.time.LocalDate.of(2024, 5, 24)), + () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 50), + () -> assertThat(row.getInt("mediumint_column")).isEqualTo(1000), + () -> assertThat(row.getInt("int_column")).isEqualTo(50000), + () -> assertThat(row.getLong("bigint_column")).isEqualTo(987654321L), + () -> { + float expectedFloat = 45.67f; + float actualFloat = row.getFloat("float_column"); + assertThat(Math.abs(actualFloat - expectedFloat)).isLessThan(0.001f); + }, + () -> { + double expectedDouble = 123.789; + double actualDouble = row.getDouble("double_column"); + assertThat(Math.abs(actualDouble - expectedDouble)).isLessThan(0.001); + }, + () -> assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("1234.56")), + () -> + assertThat(row.getInstant("datetime_column")) + .isEqualTo(java.time.Instant.parse("2024-02-08T08:15:30Z")), + () -> + assertThat(row.getInstant("timestamp_column")) + .isEqualTo(java.time.Instant.parse("2024-02-08T08:15:30Z")), + () -> + assertThat(row.getLocalTime("time_column")) + .isEqualTo(java.time.LocalTime.of(14, 30, 0)), + () -> assertThat(row.getString("year_column")).isEqualTo("2022"), + () -> assertThat(row.getString("char_column")).isEqualTo("char_col"), + () -> assertThat(row.getString("tinytext_column")).isEqualTo("tinytext_column_value"), + () -> assertThat(row.getString("mediumtext_column")).isEqualTo("mediumtext_column_value"), + () -> assertThat(row.getString("longtext_column")).isEqualTo("longtext_column_value"), + () -> assertThat(row.getString("enum_column")).isEqualTo("2"), + () -> assertThat(row.getBoolean("bool_column")).isFalse(), + () -> assertThat(row.getBoolean("other_bool_column")).isTrue()); + } +} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java new file mode 100644 index 0000000000..67e09e173d --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java @@ -0,0 +1,223 @@ +/* + * Copyright (C) 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package com.google.cloud.teleport.v2.templates; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; + +import com.datastax.oss.driver.api.core.cql.ResultSet; +import com.datastax.oss.driver.api.core.cql.Row; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Options; +import com.google.cloud.spanner.TransactionRunner; +import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; +import com.google.cloud.teleport.metadata.TemplateIntegrationTest; +import com.google.pubsub.v1.SubscriptionName; +import java.io.IOException; +import java.time.Duration; +import java.util.HashSet; +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.pubsub.PubsubResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.storage.GcsResourceManager; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) +@TemplateIntegrationTest(SpannerToSourceDb.class) +@RunWith(JUnit4.class) +public class SpannerToSourceDbCassandraIT extends SpannerToCassandraDbITBase { + + private static final Logger LOG = LoggerFactory.getLogger(SpannerToSourceDbCassandraIT.class); + + private static final String SPANNER_DDL_RESOURCE = + "SpannerToCassandraSourceIT/spanner-schema.sql"; + private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = + "SpannerToCassandraSourceIT/cassandra-schema.sql"; + + private static final String TABLE = "Users"; + private static final HashSet testInstances = new HashSet<>(); + private static PipelineLauncher.LaunchInfo jobInfo; + public static SpannerResourceManager spannerResourceManager; + private static SpannerResourceManager spannerMetadataResourceManager; + public static CassandraSharedResourceManager cassandraResourceManager; + private static GcsResourceManager gcsResourceManager; + private static PubsubResourceManager pubsubResourceManager; + private SubscriptionName subscriptionName; + + /** + * Setup resource managers and Launch dataflow job once during the execution of this test class. + * + * @throws IOException + */ + @Before + public void setUp() throws IOException { + skipBaseCleanup = true; + synchronized (SpannerToSourceDbCassandraIT.class) { + testInstances.add(this); + if (jobInfo == null) { + spannerResourceManager = createSpannerDatabase(SPANNER_DDL_RESOURCE); + spannerMetadataResourceManager = createSpannerMetadataDatabase(); + + cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); + gcsResourceManager = + GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) + .build(); + createAndUploadCassandraConfigToGcs(gcsResourceManager, cassandraResourceManager); + createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); + pubsubResourceManager = setUpPubSubResourceManager(); + subscriptionName = + createPubsubResources( + getClass().getSimpleName(), + pubsubResourceManager, + getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); + jobInfo = + launchDataflowJob( + gcsResourceManager, + spannerResourceManager, + spannerMetadataResourceManager, + subscriptionName.toString(), + null, + null, + null, + null, + null); + } + } + } + + /** + * Cleanup dataflow job and all the resources and resource managers. + * + * @throws IOException + */ + @AfterClass + public static void cleanUp() throws IOException { + for (SpannerToSourceDbCassandraIT instance : testInstances) { + instance.tearDownBase(); + } + ResourceManagerUtils.cleanResources( + spannerResourceManager, + cassandraResourceManager, + spannerMetadataResourceManager, + gcsResourceManager, + pubsubResourceManager); + } + + @Test + public void spannerToCasandraSourceDbBasic() throws InterruptedException, IOException { + assertThatPipeline(jobInfo).isRunning(); + writeRowInSpanner(); + assertRowInCassandraDB(); + } + + private long getRowCount() { + String query = String.format("SELECT COUNT(*) FROM %s", TABLE); + ResultSet resultSet = cassandraResourceManager.executeStatement(query); + Row row = resultSet.one(); + if (row != null) { + return row.getLong(0); + } else { + throw new RuntimeException("Query did not return a result for table: " + TABLE); + } + } + + private void writeRowInSpanner() { + Mutation m1 = + Mutation.newInsertOrUpdateBuilder("users") + .set("id") + .to(1) + .set("full_name") + .to("A") + .set("from") + .to("B") + .build(); + spannerResourceManager.write(m1); + + Mutation m2 = + Mutation.newInsertOrUpdateBuilder("users2") + .set("id") + .to(2) + .set("full_name") + .to("BB") + .build(); + spannerResourceManager.write(m2); + + // Write a single record to Spanner for the given logical shard + // Add the record with the transaction tag as txBy= + SpannerConfig spannerConfig = + SpannerConfig.create() + .withProjectId(PROJECT) + .withInstanceId(spannerResourceManager.getInstanceId()) + .withDatabaseId(spannerResourceManager.getDatabaseId()); + SpannerAccessor spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); + spannerAccessor + .getDatabaseClient() + .readWriteTransaction( + Options.tag("txBy=forwardMigration"), + Options.priority(spannerConfig.getRpcPriority().get())) + .run( + (TransactionRunner.TransactionCallable) + transaction -> { + Mutation m3 = + Mutation.newInsertOrUpdateBuilder("users") + .set("id") + .to(3) + .set("full_name") + .to("GG") + .set("from") + .to("BB") + .build(); + transaction.buffer(m3); + return null; + }); + } + + private void assertRowInCassandraDB() throws InterruptedException { + PipelineOperator.Result result = + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); + assertThatResult(result).meetsConditions(); + Iterable rows; + try { + LOG.info("Reading from Cassandra table: {}", TABLE); + rows = cassandraResourceManager.readTable(TABLE); + LOG.info("Cassandra Rows: {}", rows.toString()); + } catch (Exception e) { + throw new RuntimeException("Failed to read from Cassandra table: " + TABLE, e); + } + + assertThat(rows).hasSize(1); + + Row row = rows.iterator().next(); + LOG.info("Cassandra Row to Assert: {}", row.toString()); + assertThat(row.getInt("id")).isEqualTo(1); + assertThat(row.getString("full_name")).isEqualTo("A"); + assertThat(row.getString("from")).isEqualTo("B"); + } +} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java index 590187fc99..f6fcbc1812 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java @@ -251,6 +251,40 @@ public void testGetColumnValueByTypeForStringHex() { assertArrayEquals(expectedBytes, actualBytes); } + @Test + public void testGetColumnValueByTypeForStringByteArrayBase64Encode() { + SpannerColumnType spannerType = new SpannerColumnType("bytes", false); + SourceColumnType sourceColumnType = new SourceColumnType("blob", null, null); + String columnName = "lastName"; + byte[] expectedBytes = new byte[] {1, 2, 3, 4, 5}; + + String columnValue = java.util.Base64.getEncoder().encodeToString(expectedBytes); + String sourceDbTimezoneOffset = null; + + SpannerColumnDefinition spannerColDef = new SpannerColumnDefinition(columnName, spannerType); + SourceColumnDefinition sourceColDef = new SourceColumnDefinition(columnName, sourceColumnType); + + JSONObject valuesJson = new JSONObject(); + valuesJson.put(columnName, columnValue); + + PreparedStatementValueObject result = + getColumnValueByType(spannerColDef, sourceColDef, valuesJson, sourceDbTimezoneOffset); + + Object castResult = CassandraTypeHandler.castToExpectedType(result.dataType(), result.value()); + + byte[] actualBytes; + if (castResult instanceof ByteBuffer) { + ByteBuffer byteBuffer = (ByteBuffer) castResult; + actualBytes = new byte[byteBuffer.remaining()]; + byteBuffer.get(actualBytes); + } else if (castResult instanceof byte[]) { + actualBytes = (byte[]) castResult; + } else { + throw new AssertionError("Unexpected type for castResult"); + } + assertArrayEquals(expectedBytes, actualBytes); + } + @Test public void testGetColumnValueByTypeForBlobEncodeInStringHexToBlob() { SpannerColumnType spannerType = new SpannerColumnType("bytes", false); diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf new file mode 100644 index 0000000000..97486a8de0 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf @@ -0,0 +1,12 @@ + # Configuration for the DataStax Java driver for Apache Cassandra®. + # This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md. + # This file is meant to be used only in unit tests to test loading configuration from file. + # DO NOT USE FOR PRODUCTION. + + datastax-java-driver { + basic.contact-points = ["##host##:##port##"] + basic.session-keyspace = "##keyspace##" + basic.load-balancing-policy { + local-datacenter = "datacenter1" + } + } diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql new file mode 100644 index 0000000000..366cf32d2b --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql @@ -0,0 +1,43 @@ +CREATE TABLE AllDatatypeColumns ( + varchar_column text PRIMARY KEY, + tinyint_column tinyint, + text_column text, + date_column date, + smallint_column smallint, + mediumint_column int, + int_column int, + bigint_column bigint, + float_column float, + double_column double, + decimal_column decimal, + datetime_column timestamp, + timestamp_column timestamp, + time_column time, + year_column text, + char_column text, + tinytext_column text, + mediumtext_column text, + longtext_column text, + enum_column text, + bool_column boolean, + other_bool_column boolean, + bytes_column BLOB, + list_text_column list, + list_int_column list, + frozen_list_bigint_column frozen>, + set_text_column set, + set_date_column set, + frozen_set_bool_column frozen>, + map_text_to_int_column map, + map_date_to_text_column map, + frozen_map_int_to_bool_column frozen>, + map_text_to_list_column map>>, + map_text_to_set_column map>>, + set_of_maps_column set>>, + list_of_sets_column list>>, + frozen_map_text_to_list_column map>>, + frozen_map_text_to_set_column map>>, + frozen_set_of_maps_column set>>, + frozen_list_of_sets_column list>>, + varint_column varint +); \ No newline at end of file diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql new file mode 100644 index 0000000000..4cb32f2778 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql @@ -0,0 +1,49 @@ +CREATE TABLE IF NOT EXISTS alldatatypecolumns ( + varchar_column STRING(20) NOT NULL, + tinyint_column INT64, + text_column STRING(MAX), + date_column DATE, + smallint_column INT64, + mediumint_column INT64, + int_column INT64, + bigint_column INT64, + float_column FLOAT64, + double_column FLOAT64, + decimal_column NUMERIC, + datetime_column TIMESTAMP, + timestamp_column TIMESTAMP, + time_column STRING(MAX), + year_column STRING(MAX), + char_column STRING(10), + tinytext_column STRING(MAX), + mediumtext_column STRING(MAX), + longtext_column STRING(MAX), + enum_column STRING(MAX), + bool_column BOOL, + other_bool_column BOOL, + bytes_column BYTES(MAX), + list_text_column JSON, + list_int_column JSON, + frozen_list_bigint_column JSON, + set_text_column JSON, + set_date_column JSON, + frozen_set_bool_column JSON, + map_text_to_int_column JSON, + map_date_to_text_column JSON, + frozen_map_int_to_bool_column JSON, + map_text_to_list_column JSON, + map_text_to_set_column JSON, + set_of_maps_column JSON, + list_of_sets_column JSON, + frozen_map_text_to_list_column JSON, + frozen_map_text_to_set_column JSON, + frozen_set_of_maps_column JSON, + frozen_list_of_sets_column JSON, + varint_column BYTES(MAX) +) PRIMARY KEY(varchar_column); + +CREATE CHANGE STREAM allstream + FOR ALL OPTIONS ( + value_capture_type = 'NEW_ROW', + retention_period = '7d' + ); \ No newline at end of file diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-config-template.conf b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-config-template.conf new file mode 100644 index 0000000000..97486a8de0 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-config-template.conf @@ -0,0 +1,12 @@ + # Configuration for the DataStax Java driver for Apache Cassandra®. + # This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md. + # This file is meant to be used only in unit tests to test loading configuration from file. + # DO NOT USE FOR PRODUCTION. + + datastax-java-driver { + basic.contact-points = ["##host##:##port##"] + basic.session-keyspace = "##keyspace##" + basic.load-balancing-policy { + local-datacenter = "datacenter1" + } + } diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql new file mode 100644 index 0000000000..e220267733 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql @@ -0,0 +1,5 @@ +CREATE TABLE users ( + id int PRIMARY KEY, + full_name text, + "from" text +); \ No newline at end of file diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql new file mode 100644 index 0000000000..5cedd597e1 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql @@ -0,0 +1,16 @@ +CREATE TABLE IF NOT EXISTS users ( + id INT64 NOT NULL, + full_name STRING(25), + `from` STRING(25) +) PRIMARY KEY(id); + +CREATE TABLE IF NOT EXISTS users2 ( + id INT64 NOT NULL, + full_name STRING(25), + ) PRIMARY KEY(id); + +CREATE CHANGE STREAM allstream + FOR ALL OPTIONS ( + value_capture_type = 'NEW_ROW', + retention_period = '7d' +); \ No newline at end of file From e93f219f7be1f55e0461dbcaa3fe244da79a1abf Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Tue, 28 Jan 2025 23:41:37 +0530 Subject: [PATCH 02/19] Added IT Fixes --- .../SpannerToCassandraSourceDbDatatypeIT.java | 413 +++++++++++------- 1 file changed, 261 insertions(+), 152 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java index 4634653bc5..cb3e822492 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java @@ -28,19 +28,26 @@ import com.google.cloud.spanner.Value; import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; import com.google.cloud.teleport.metadata.TemplateIntegrationTest; +import com.google.common.io.Resources; import com.google.pubsub.v1.SubscriptionName; import java.io.IOException; import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; 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.pubsub.PubsubResourceManager; import org.apache.beam.it.gcp.spanner.SpannerResourceManager; import org.apache.beam.it.gcp.storage.GcsResourceManager; +import org.apache.commons.codec.DecoderException; +import org.apache.commons.codec.binary.Hex; import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; @@ -55,13 +62,15 @@ public class SpannerToCassandraSourceDbDatatypeIT extends SpannerToCassandraDbITBase { private static final String SPANNER_DDL_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; + "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; + private static final String SESSION_FILE_RESOURCE = + "SpannerToCassandraSourceDbDatatypeIT/session.json"; private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; + "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; private static final String TABLE = "AllDatatypeColumns"; private static final HashSet testInstances = - new HashSet<>(); + new HashSet<>(); private static PipelineLauncher.LaunchInfo jobInfo; public static SpannerResourceManager spannerResourceManager; private static SpannerResourceManager spannerMetadataResourceManager; @@ -87,27 +96,29 @@ public void setUp() throws IOException { cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); gcsResourceManager = - GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) - .build(); + GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) + .build(); createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); createAndUploadCassandraConfigToGcs(gcsResourceManager, cassandraResourceManager); + gcsResourceManager.uploadArtifact( + "input/session.json", Resources.getResource(SESSION_FILE_RESOURCE).getPath()); pubsubResourceManager = setUpPubSubResourceManager(); subscriptionName = - createPubsubResources( - getClass().getSimpleName(), - pubsubResourceManager, - getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); + createPubsubResources( + getClass().getSimpleName(), + pubsubResourceManager, + getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); jobInfo = - launchDataflowJob( - gcsResourceManager, - spannerResourceManager, - spannerMetadataResourceManager, - subscriptionName.toString(), - null, - null, - null, - null, - null); + launchDataflowJob( + gcsResourceManager, + spannerResourceManager, + spannerMetadataResourceManager, + subscriptionName.toString(), + null, + null, + null, + null, + null); } } } @@ -123,16 +134,16 @@ public static void cleanUp() throws IOException { instance.tearDownBase(); } ResourceManagerUtils.cleanResources( - spannerResourceManager, - cassandraResourceManager, - spannerMetadataResourceManager, - gcsResourceManager, - pubsubResourceManager); + spannerResourceManager, + cassandraResourceManager, + spannerMetadataResourceManager, + gcsResourceManager, + pubsubResourceManager); } @Test public void spannerToCassandraSourceDataTypeConversionTest() - throws InterruptedException, IOException, MultipleFailureException { + throws InterruptedException, IOException, MultipleFailureException { assertThatPipeline(jobInfo).isRunning(); writeRowInSpanner(); assertRowInCassandraDB(); @@ -151,92 +162,92 @@ private long getRowCount() { private void writeRowInSpanner() { Mutation mutation = - Mutation.newInsertOrUpdateBuilder(TABLE) - .set("varchar_column") - .to("SampleVarchar") - .set("tinyint_column") - .to(127) - .set("text_column") - .to("This is some sample text data for the text column.") - .set("date_column") - .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) - .set("smallint_column") - .to(32767) - .set("mediumint_column") - .to(8388607) - .set("int_column") - .to(2147483647) - .set("bigint_column") - .to(9223372036854775807L) - .set("float_column") - .to(3.14159) - .set("double_column") - .to(2.718281828459045) - .set("decimal_column") - .to(new BigDecimal("12345.6789")) - .set("datetime_column") - .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) - .set("timestamp_column") - .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) - .set("time_column") - .to("12:30:00") - .set("year_column") - .to("2025") - .set("char_column") - .to("CHAR_DATA") - .set("tinytext_column") - .to("Short text for tinytext.") - .set("mediumtext_column") - .to("Longer text data for mediumtext column.") - .set("longtext_column") - .to("Very long text data that exceeds the medium text column length for long text.") - .set("enum_column") - .to("OptionA") - .set("bool_column") - .to(Value.bool(Boolean.TRUE)) - .set("other_bool_column") - .to(Value.bool(Boolean.FALSE)) - .set("bytes_column") - .to(Value.bytes(ByteArray.copyFrom("SGVsbG8gd29ybGQ=".getBytes()))) - .set("list_text_column") - .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) - .set("list_int_column") - .to(Value.json("[1, 2, 3, 4, 5]")) - .set("frozen_list_bigint_column") - .to(Value.json("[123456789012345, 987654321012345]")) - .set("set_text_column") - .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) - .set("set_date_column") - .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) - .set("frozen_set_bool_column") - .to(Value.json("[true, false]")) - .set("map_text_to_int_column") - .to(Value.json("{\"key1\": 10, \"key2\": 20}")) - .set("map_date_to_text_column") - .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) - .set("frozen_map_int_to_bool_column") - .to(Value.json("{\"1\": true, \"2\": false}")) - .set("map_text_to_list_column") - .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) - .set("map_text_to_set_column") - .to( - Value.json( - "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) - .set("set_of_maps_column") - .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) - .set("list_of_sets_column") - .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) - .set("frozen_map_text_to_list_column") - .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) - .set("frozen_map_text_to_set_column") - .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) - .set("frozen_set_of_maps_column") - .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) - .set("frozen_list_of_sets_column") - .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) - .set("varint_column") - .to(Value.bytes(ByteArray.copyFrom("b3f5ed4f".getBytes()))) - .build(); + Mutation.newInsertOrUpdateBuilder(TABLE) + .set("varchar_column") + .to("SampleVarchar") + .set("tinyint_column") + .to(127) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(32767) + .set("mediumint_column") + .to(8388607) + .set("int_column") + .to(2147483647) + .set("bigint_column") + .to(9223372036854775807L) + .set("float_column") + .to(3.14159) + .set("double_column") + .to(2.718281828459045) + .set("decimal_column") + .to(new BigDecimal("12345.6789")) + .set("datetime_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(Value.bool(Boolean.TRUE)) + .set("other_bool_column") + .to(Value.bool(Boolean.FALSE)) + .set("bytes_column") + .to(Value.bytes(ByteArray.copyFrom("SGVsbG8gd29ybGQ=".getBytes()))) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to(Value.bytes(ByteArray.copyFrom("b3f5ed4f".getBytes()))) + .build(); spannerResourceManager.write(mutation); } @@ -256,9 +267,9 @@ private void assertAll(Runnable... assertions) throws MultipleFailureException { private void assertRowInCassandraDB() throws InterruptedException, MultipleFailureException { PipelineOperator.Result result = - pipelineOperator() - .waitForCondition( - createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); assertThatResult(result).meetsConditions(); Iterable rows; try { @@ -274,43 +285,141 @@ private void assertRowInCassandraDB() throws InterruptedException, MultipleFailu assertThat(rows).hasSize(1); assertAll( - () -> assertThat(row.getString("varchar_column")).isEqualTo("value1"), - () -> assertThat(row.getInt("tinyint_column")).isEqualTo(10), - () -> assertThat(row.getString("text_column")).isEqualTo("text_column_value"), - () -> - assertThat(row.getLocalDate("date_column")) - .isEqualTo(java.time.LocalDate.of(2024, 5, 24)), - () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 50), - () -> assertThat(row.getInt("mediumint_column")).isEqualTo(1000), - () -> assertThat(row.getInt("int_column")).isEqualTo(50000), - () -> assertThat(row.getLong("bigint_column")).isEqualTo(987654321L), - () -> { - float expectedFloat = 45.67f; - float actualFloat = row.getFloat("float_column"); - assertThat(Math.abs(actualFloat - expectedFloat)).isLessThan(0.001f); - }, - () -> { - double expectedDouble = 123.789; - double actualDouble = row.getDouble("double_column"); - assertThat(Math.abs(actualDouble - expectedDouble)).isLessThan(0.001); - }, - () -> assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("1234.56")), - () -> - assertThat(row.getInstant("datetime_column")) - .isEqualTo(java.time.Instant.parse("2024-02-08T08:15:30Z")), - () -> - assertThat(row.getInstant("timestamp_column")) - .isEqualTo(java.time.Instant.parse("2024-02-08T08:15:30Z")), - () -> - assertThat(row.getLocalTime("time_column")) - .isEqualTo(java.time.LocalTime.of(14, 30, 0)), - () -> assertThat(row.getString("year_column")).isEqualTo("2022"), - () -> assertThat(row.getString("char_column")).isEqualTo("char_col"), - () -> assertThat(row.getString("tinytext_column")).isEqualTo("tinytext_column_value"), - () -> assertThat(row.getString("mediumtext_column")).isEqualTo("mediumtext_column_value"), - () -> assertThat(row.getString("longtext_column")).isEqualTo("longtext_column_value"), - () -> assertThat(row.getString("enum_column")).isEqualTo("2"), - () -> assertThat(row.getBoolean("bool_column")).isFalse(), - () -> assertThat(row.getBoolean("other_bool_column")).isTrue()); + // Basic Data Types + () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), + () -> assertThat(row.getLong("bigint_column")).isEqualTo(9223372036854775807L), + () -> assertThat(row.getBoolean("bool_column")).isTrue(), + () -> { + String hexString = "5347567362473867643239796247513d"; + byte[] byteArray; + try { + byteArray = Hex.decodeHex(hexString); + } catch (DecoderException e) { + byteArray = new byte[0]; + } + ByteBuffer expectedBuffer = ByteBuffer.wrap(byteArray); + assertThat(row.getByteBuffer("bytes_column")).isEqualTo(expectedBuffer); + }, + () -> assertThat(row.getString("char_column")).isEqualTo("CHAR_DATA"), + () -> + assertThat(row.getLocalDate("date_column")) + .isEqualTo(java.time.LocalDate.of(2025, 1, 27)), + () -> + assertThat(row.getInstant("datetime_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("12345.6789")), + () -> assertThat(row.getDouble("double_column")).isEqualTo(2.718281828459045), + () -> assertThat(row.getFloat("float_column")).isEqualTo(3.14159f), + + // Collections (frozen, list, set, map) + () -> + assertThat(row.getList("frozen_list_bigint_column", Long.class)) + .isEqualTo(Arrays.asList(123456789012345L, 987654321012345L)), + () -> + assertThat(row.getSet("frozen_set_bool_column", Boolean.class)) + .isEqualTo(new HashSet<>(Arrays.asList(false, true))), + () -> + assertThat(row.getMap("frozen_map_int_to_bool_column", Integer.class, Boolean.class)) + .isEqualTo(Map.of(1, true, 2, false)), + () -> + assertThat(row.getMap("frozen_map_text_to_list_column", String.class, List.class)) + .isEqualTo(Map.of("fruits", Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("frozen_map_text_to_set_column", String.class, Set.class)) + .isEqualTo(Map.of("vegetables", new HashSet<>(Arrays.asList("carrot", "spinach")))), + () -> + assertThat(row.getSet("frozen_set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + + // Lists and Sets + () -> + assertThat(row.getList("list_int_column", Integer.class)) + .isEqualTo(Arrays.asList(1, 2, 3, 4, 5)), + () -> + assertThat(row.getList("list_text_column", String.class)) + .isEqualTo(Arrays.asList("apple", "banana", "cherry")), + () -> + assertThat(row.getList("list_of_sets_column", Set.class)) + .isEqualTo( + Arrays.asList( + new HashSet<>(Arrays.asList("apple", "banana")), + new HashSet<>(Arrays.asList("carrot", "spinach")))), + + // Maps + () -> + assertThat( + row.getMap("map_date_to_text_column", java.time.LocalDate.class, String.class)) + .isEqualTo( + Map.of( + java.time.LocalDate.parse("2025-01-27"), "event1", + java.time.LocalDate.parse("2025-02-01"), "event2")), + () -> + assertThat(row.getMap("map_text_to_int_column", String.class, Integer.class)) + .isEqualTo(Map.of("key1", 10, "key2", 20)), + () -> + assertThat(row.getMap("map_text_to_list_column", String.class, List.class)) + .isEqualTo( + Map.of( + "color", + Arrays.asList("red", "green"), + "fruit", + Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("map_text_to_set_column", String.class, Set.class)) + .isEqualTo( + Map.of( + "fruit", + new HashSet<>(Arrays.asList("apple", "banana")), + "vegetables", + new HashSet<>(Arrays.asList("carrot", "spinach")))), + + // Sets + () -> + assertThat(row.getSet("set_date_column", java.time.LocalDate.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + java.time.LocalDate.parse("2025-01-27"), + java.time.LocalDate.parse("2025-02-01")))), + () -> + assertThat(row.getSet("set_text_column", String.class)) + .isEqualTo(new HashSet<>(Arrays.asList("apple", "orange", "banana"))), + () -> + assertThat(row.getSet("set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + + // Other Basic Types + () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 32767), + () -> assertThat(row.getInt("mediumint_column")).isEqualTo(8388607), + () -> assertThat(row.getInt("int_column")).isEqualTo(2147483647), + () -> assertThat(row.getString("enum_column")).isEqualTo("OptionA"), + () -> assertThat(row.getString("year_column")).isEqualTo("2025"), + () -> + assertThat(row.getString("longtext_column")) + .isEqualTo( + "Very long text data that exceeds the medium text column length for long text."), + () -> assertThat(row.getString("tinytext_column")).isEqualTo("Short text for tinytext."), + () -> + assertThat(row.getString("mediumtext_column")) + .isEqualTo("Longer text data for mediumtext column."), + () -> + assertThat(row.getString("text_column")) + .isEqualTo("This is some sample text data for the text column."), + () -> + assertThat(row.getLocalTime("time_column")) + .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), + () -> + assertThat(row.getInstant("timestamp_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigInteger("varint_column")) + .isEqualTo(java.math.BigInteger.valueOf(7076111819049546854L))); } } From f767b4f76c738b9c016e626c1695fceae22ca078 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Tue, 28 Jan 2025 23:43:18 +0530 Subject: [PATCH 03/19] Added IT fixes --- .../SpannerToCassandraSourceDbDatatypeIT.java | 495 +++++++++--------- .../cassandra-schema.sql | 6 +- 2 files changed, 248 insertions(+), 253 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java index cb3e822492..b279641ff8 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java @@ -28,7 +28,6 @@ import com.google.cloud.spanner.Value; import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; import com.google.cloud.teleport.metadata.TemplateIntegrationTest; -import com.google.common.io.Resources; import com.google.pubsub.v1.SubscriptionName; import java.io.IOException; import java.math.BigDecimal; @@ -62,15 +61,13 @@ public class SpannerToCassandraSourceDbDatatypeIT extends SpannerToCassandraDbITBase { private static final String SPANNER_DDL_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; - private static final String SESSION_FILE_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/session.json"; + "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; + "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; private static final String TABLE = "AllDatatypeColumns"; private static final HashSet testInstances = - new HashSet<>(); + new HashSet<>(); private static PipelineLauncher.LaunchInfo jobInfo; public static SpannerResourceManager spannerResourceManager; private static SpannerResourceManager spannerMetadataResourceManager; @@ -96,29 +93,27 @@ public void setUp() throws IOException { cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); gcsResourceManager = - GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) - .build(); + GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) + .build(); createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); createAndUploadCassandraConfigToGcs(gcsResourceManager, cassandraResourceManager); - gcsResourceManager.uploadArtifact( - "input/session.json", Resources.getResource(SESSION_FILE_RESOURCE).getPath()); pubsubResourceManager = setUpPubSubResourceManager(); subscriptionName = - createPubsubResources( - getClass().getSimpleName(), - pubsubResourceManager, - getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); + createPubsubResources( + getClass().getSimpleName(), + pubsubResourceManager, + getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); jobInfo = - launchDataflowJob( - gcsResourceManager, - spannerResourceManager, - spannerMetadataResourceManager, - subscriptionName.toString(), - null, - null, - null, - null, - null); + launchDataflowJob( + gcsResourceManager, + spannerResourceManager, + spannerMetadataResourceManager, + subscriptionName.toString(), + null, + null, + null, + null, + null); } } } @@ -134,16 +129,16 @@ public static void cleanUp() throws IOException { instance.tearDownBase(); } ResourceManagerUtils.cleanResources( - spannerResourceManager, - cassandraResourceManager, - spannerMetadataResourceManager, - gcsResourceManager, - pubsubResourceManager); + spannerResourceManager, + cassandraResourceManager, + spannerMetadataResourceManager, + gcsResourceManager, + pubsubResourceManager); } @Test public void spannerToCassandraSourceDataTypeConversionTest() - throws InterruptedException, IOException, MultipleFailureException { + throws InterruptedException, IOException, MultipleFailureException { assertThatPipeline(jobInfo).isRunning(); writeRowInSpanner(); assertRowInCassandraDB(); @@ -162,92 +157,92 @@ private long getRowCount() { private void writeRowInSpanner() { Mutation mutation = - Mutation.newInsertOrUpdateBuilder(TABLE) - .set("varchar_column") - .to("SampleVarchar") - .set("tinyint_column") - .to(127) - .set("text_column") - .to("This is some sample text data for the text column.") - .set("date_column") - .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) - .set("smallint_column") - .to(32767) - .set("mediumint_column") - .to(8388607) - .set("int_column") - .to(2147483647) - .set("bigint_column") - .to(9223372036854775807L) - .set("float_column") - .to(3.14159) - .set("double_column") - .to(2.718281828459045) - .set("decimal_column") - .to(new BigDecimal("12345.6789")) - .set("datetime_column") - .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) - .set("timestamp_column") - .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) - .set("time_column") - .to("12:30:00") - .set("year_column") - .to("2025") - .set("char_column") - .to("CHAR_DATA") - .set("tinytext_column") - .to("Short text for tinytext.") - .set("mediumtext_column") - .to("Longer text data for mediumtext column.") - .set("longtext_column") - .to("Very long text data that exceeds the medium text column length for long text.") - .set("enum_column") - .to("OptionA") - .set("bool_column") - .to(Value.bool(Boolean.TRUE)) - .set("other_bool_column") - .to(Value.bool(Boolean.FALSE)) - .set("bytes_column") - .to(Value.bytes(ByteArray.copyFrom("SGVsbG8gd29ybGQ=".getBytes()))) - .set("list_text_column") - .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) - .set("list_int_column") - .to(Value.json("[1, 2, 3, 4, 5]")) - .set("frozen_list_bigint_column") - .to(Value.json("[123456789012345, 987654321012345]")) - .set("set_text_column") - .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) - .set("set_date_column") - .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) - .set("frozen_set_bool_column") - .to(Value.json("[true, false]")) - .set("map_text_to_int_column") - .to(Value.json("{\"key1\": 10, \"key2\": 20}")) - .set("map_date_to_text_column") - .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) - .set("frozen_map_int_to_bool_column") - .to(Value.json("{\"1\": true, \"2\": false}")) - .set("map_text_to_list_column") - .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) - .set("map_text_to_set_column") - .to( - Value.json( - "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) - .set("set_of_maps_column") - .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) - .set("list_of_sets_column") - .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) - .set("frozen_map_text_to_list_column") - .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) - .set("frozen_map_text_to_set_column") - .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) - .set("frozen_set_of_maps_column") - .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) - .set("frozen_list_of_sets_column") - .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) - .set("varint_column") - .to(Value.bytes(ByteArray.copyFrom("b3f5ed4f".getBytes()))) - .build(); + Mutation.newInsertOrUpdateBuilder(TABLE) + .set("varchar_column") + .to("SampleVarchar") + .set("tinyint_column") + .to(127) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(32767) + .set("mediumint_column") + .to(8388607) + .set("int_column") + .to(2147483647) + .set("bigint_column") + .to(9223372036854775807L) + .set("float_column") + .to(3.14159) + .set("double_column") + .to(2.718281828459045) + .set("decimal_column") + .to(new BigDecimal("12345.6789")) + .set("datetime_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(Value.bool(Boolean.TRUE)) + .set("other_bool_column") + .to(Value.bool(Boolean.FALSE)) + .set("bytes_column") + .to(Value.bytes(ByteArray.copyFrom("SGVsbG8gd29ybGQ=".getBytes()))) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to(Value.bytes(ByteArray.copyFrom("b3f5ed4f".getBytes()))) + .build(); spannerResourceManager.write(mutation); } @@ -267,9 +262,9 @@ private void assertAll(Runnable... assertions) throws MultipleFailureException { private void assertRowInCassandraDB() throws InterruptedException, MultipleFailureException { PipelineOperator.Result result = - pipelineOperator() - .waitForCondition( - createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); assertThatResult(result).meetsConditions(); Iterable rows; try { @@ -285,141 +280,141 @@ private void assertRowInCassandraDB() throws InterruptedException, MultipleFailu assertThat(rows).hasSize(1); assertAll( - // Basic Data Types - () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), - () -> assertThat(row.getLong("bigint_column")).isEqualTo(9223372036854775807L), - () -> assertThat(row.getBoolean("bool_column")).isTrue(), - () -> { - String hexString = "5347567362473867643239796247513d"; - byte[] byteArray; - try { - byteArray = Hex.decodeHex(hexString); - } catch (DecoderException e) { - byteArray = new byte[0]; - } - ByteBuffer expectedBuffer = ByteBuffer.wrap(byteArray); - assertThat(row.getByteBuffer("bytes_column")).isEqualTo(expectedBuffer); - }, - () -> assertThat(row.getString("char_column")).isEqualTo("CHAR_DATA"), - () -> - assertThat(row.getLocalDate("date_column")) - .isEqualTo(java.time.LocalDate.of(2025, 1, 27)), - () -> - assertThat(row.getInstant("datetime_column")) - .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), - () -> - assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("12345.6789")), - () -> assertThat(row.getDouble("double_column")).isEqualTo(2.718281828459045), - () -> assertThat(row.getFloat("float_column")).isEqualTo(3.14159f), + // Basic Data Types + () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), + () -> assertThat(row.getLong("bigint_column")).isEqualTo(9223372036854775807L), + () -> assertThat(row.getBoolean("bool_column")).isTrue(), + () -> { + String hexString = "5347567362473867643239796247513d"; + byte[] byteArray; + try { + byteArray = Hex.decodeHex(hexString); + } catch (DecoderException e) { + byteArray = new byte[0]; + } + ByteBuffer expectedBuffer = ByteBuffer.wrap(byteArray); + assertThat(row.getByteBuffer("bytes_column")).isEqualTo(expectedBuffer); + }, + () -> assertThat(row.getString("char_column")).isEqualTo("CHAR_DATA"), + () -> + assertThat(row.getLocalDate("date_column")) + .isEqualTo(java.time.LocalDate.of(2025, 1, 27)), + () -> + assertThat(row.getInstant("datetime_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("12345.6789")), + () -> assertThat(row.getDouble("double_column")).isEqualTo(2.718281828459045), + () -> assertThat(row.getFloat("float_column")).isEqualTo(3.14159f), - // Collections (frozen, list, set, map) - () -> - assertThat(row.getList("frozen_list_bigint_column", Long.class)) - .isEqualTo(Arrays.asList(123456789012345L, 987654321012345L)), - () -> - assertThat(row.getSet("frozen_set_bool_column", Boolean.class)) - .isEqualTo(new HashSet<>(Arrays.asList(false, true))), - () -> - assertThat(row.getMap("frozen_map_int_to_bool_column", Integer.class, Boolean.class)) - .isEqualTo(Map.of(1, true, 2, false)), - () -> - assertThat(row.getMap("frozen_map_text_to_list_column", String.class, List.class)) - .isEqualTo(Map.of("fruits", Arrays.asList("apple", "banana"))), - () -> - assertThat(row.getMap("frozen_map_text_to_set_column", String.class, Set.class)) - .isEqualTo(Map.of("vegetables", new HashSet<>(Arrays.asList("carrot", "spinach")))), - () -> - assertThat(row.getSet("frozen_set_of_maps_column", Map.class)) - .isEqualTo( - new HashSet<>( - Arrays.asList( - Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + // Collections (frozen, list, set, map) + () -> + assertThat(row.getList("frozen_list_bigint_column", Long.class)) + .isEqualTo(Arrays.asList(123456789012345L, 987654321012345L)), + () -> + assertThat(row.getSet("frozen_set_bool_column", Boolean.class)) + .isEqualTo(new HashSet<>(Arrays.asList(false, true))), + () -> + assertThat(row.getMap("frozen_map_int_to_bool_column", Integer.class, Boolean.class)) + .isEqualTo(Map.of(1, true, 2, false)), + () -> + assertThat(row.getMap("frozen_map_text_to_list_column", String.class, List.class)) + .isEqualTo(Map.of("fruits", Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("frozen_map_text_to_set_column", String.class, Set.class)) + .isEqualTo(Map.of("vegetables", new HashSet<>(Arrays.asList("carrot", "spinach")))), + () -> + assertThat(row.getSet("frozen_set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), - // Lists and Sets - () -> - assertThat(row.getList("list_int_column", Integer.class)) - .isEqualTo(Arrays.asList(1, 2, 3, 4, 5)), - () -> - assertThat(row.getList("list_text_column", String.class)) - .isEqualTo(Arrays.asList("apple", "banana", "cherry")), - () -> - assertThat(row.getList("list_of_sets_column", Set.class)) - .isEqualTo( - Arrays.asList( - new HashSet<>(Arrays.asList("apple", "banana")), - new HashSet<>(Arrays.asList("carrot", "spinach")))), + // Lists and Sets + () -> + assertThat(row.getList("list_int_column", Integer.class)) + .isEqualTo(Arrays.asList(1, 2, 3, 4, 5)), + () -> + assertThat(row.getList("list_text_column", String.class)) + .isEqualTo(Arrays.asList("apple", "banana", "cherry")), + () -> + assertThat(row.getList("list_of_sets_column", Set.class)) + .isEqualTo( + Arrays.asList( + new HashSet<>(Arrays.asList("apple", "banana")), + new HashSet<>(Arrays.asList("carrot", "spinach")))), - // Maps - () -> - assertThat( - row.getMap("map_date_to_text_column", java.time.LocalDate.class, String.class)) - .isEqualTo( - Map.of( - java.time.LocalDate.parse("2025-01-27"), "event1", - java.time.LocalDate.parse("2025-02-01"), "event2")), - () -> - assertThat(row.getMap("map_text_to_int_column", String.class, Integer.class)) - .isEqualTo(Map.of("key1", 10, "key2", 20)), - () -> - assertThat(row.getMap("map_text_to_list_column", String.class, List.class)) - .isEqualTo( - Map.of( - "color", - Arrays.asList("red", "green"), - "fruit", - Arrays.asList("apple", "banana"))), - () -> - assertThat(row.getMap("map_text_to_set_column", String.class, Set.class)) - .isEqualTo( - Map.of( - "fruit", - new HashSet<>(Arrays.asList("apple", "banana")), - "vegetables", - new HashSet<>(Arrays.asList("carrot", "spinach")))), + // Maps + () -> + assertThat( + row.getMap("map_date_to_text_column", java.time.LocalDate.class, String.class)) + .isEqualTo( + Map.of( + java.time.LocalDate.parse("2025-01-27"), "event1", + java.time.LocalDate.parse("2025-02-01"), "event2")), + () -> + assertThat(row.getMap("map_text_to_int_column", String.class, Integer.class)) + .isEqualTo(Map.of("key1", 10, "key2", 20)), + () -> + assertThat(row.getMap("map_text_to_list_column", String.class, List.class)) + .isEqualTo( + Map.of( + "color", + Arrays.asList("red", "green"), + "fruit", + Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("map_text_to_set_column", String.class, Set.class)) + .isEqualTo( + Map.of( + "fruit", + new HashSet<>(Arrays.asList("apple", "banana")), + "vegetables", + new HashSet<>(Arrays.asList("carrot", "spinach")))), - // Sets - () -> - assertThat(row.getSet("set_date_column", java.time.LocalDate.class)) - .isEqualTo( - new HashSet<>( - Arrays.asList( - java.time.LocalDate.parse("2025-01-27"), - java.time.LocalDate.parse("2025-02-01")))), - () -> - assertThat(row.getSet("set_text_column", String.class)) - .isEqualTo(new HashSet<>(Arrays.asList("apple", "orange", "banana"))), - () -> - assertThat(row.getSet("set_of_maps_column", Map.class)) - .isEqualTo( - new HashSet<>( - Arrays.asList( - Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + // Sets + () -> + assertThat(row.getSet("set_date_column", java.time.LocalDate.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + java.time.LocalDate.parse("2025-01-27"), + java.time.LocalDate.parse("2025-02-01")))), + () -> + assertThat(row.getSet("set_text_column", String.class)) + .isEqualTo(new HashSet<>(Arrays.asList("apple", "orange", "banana"))), + () -> + assertThat(row.getSet("set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), - // Other Basic Types - () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 32767), - () -> assertThat(row.getInt("mediumint_column")).isEqualTo(8388607), - () -> assertThat(row.getInt("int_column")).isEqualTo(2147483647), - () -> assertThat(row.getString("enum_column")).isEqualTo("OptionA"), - () -> assertThat(row.getString("year_column")).isEqualTo("2025"), - () -> - assertThat(row.getString("longtext_column")) - .isEqualTo( - "Very long text data that exceeds the medium text column length for long text."), - () -> assertThat(row.getString("tinytext_column")).isEqualTo("Short text for tinytext."), - () -> - assertThat(row.getString("mediumtext_column")) - .isEqualTo("Longer text data for mediumtext column."), - () -> - assertThat(row.getString("text_column")) - .isEqualTo("This is some sample text data for the text column."), - () -> - assertThat(row.getLocalTime("time_column")) - .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), - () -> - assertThat(row.getInstant("timestamp_column")) - .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), - () -> - assertThat(row.getBigInteger("varint_column")) - .isEqualTo(java.math.BigInteger.valueOf(7076111819049546854L))); + // Other Basic Types + () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 32767), + () -> assertThat(row.getInt("mediumint_column")).isEqualTo(8388607), + () -> assertThat(row.getInt("int_column")).isEqualTo(2147483647), + () -> assertThat(row.getString("enum_column")).isEqualTo("OptionA"), + () -> assertThat(row.getString("year_column")).isEqualTo("2025"), + () -> + assertThat(row.getString("longtext_column")) + .isEqualTo( + "Very long text data that exceeds the medium text column length for long text."), + () -> assertThat(row.getString("tinytext_column")).isEqualTo("Short text for tinytext."), + () -> + assertThat(row.getString("mediumtext_column")) + .isEqualTo("Longer text data for mediumtext column."), + () -> + assertThat(row.getString("text_column")) + .isEqualTo("This is some sample text data for the text column."), + () -> + assertThat(row.getLocalTime("time_column")) + .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), + () -> + assertThat(row.getInstant("timestamp_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigInteger("varint_column")) + .isEqualTo(java.math.BigInteger.valueOf(7076111819049546854L))); } } diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql index 366cf32d2b..1a9d6a37ca 100644 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql @@ -32,12 +32,12 @@ CREATE TABLE AllDatatypeColumns ( map_date_to_text_column map, frozen_map_int_to_bool_column frozen>, map_text_to_list_column map>>, - map_text_to_set_column map>>, + map_text_to_set_column map>>, set_of_maps_column set>>, list_of_sets_column list>>, frozen_map_text_to_list_column map>>, - frozen_map_text_to_set_column map>>, + frozen_map_text_to_set_column map>>, frozen_set_of_maps_column set>>, frozen_list_of_sets_column list>>, varint_column varint -); \ No newline at end of file +); From 3869aef3903866d87b1b975970ba46c7f40a89bd Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Tue, 28 Jan 2025 23:45:56 +0530 Subject: [PATCH 04/19] removed logger --- .../v2/templates/SpannerToCassandraSourceDbDatatypeIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java index b279641ff8..cad4b6550d 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java @@ -276,7 +276,6 @@ private void assertRowInCassandraDB() throws InterruptedException, MultipleFailu assertThat(rows).hasSize(1); Row row = rows.iterator().next(); - System.out.println(row.getFormattedContents()); assertThat(rows).hasSize(1); assertAll( From da313364723c82df6b24c2b9e287acc1cfa8a045 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 29 Jan 2025 09:51:22 +0530 Subject: [PATCH 05/19] removed unwanted system log --- .../v2/templates/CassandraSharedResourceManager.java | 3 --- .../teleport/v2/templates/SpannerToCassandraDbITBase.java | 8 -------- 2 files changed, 11 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java index f60ff147ef..27c200d18d 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java @@ -154,9 +154,6 @@ public synchronized ResultSet executeStatement(String statement) { cassandraClient.execute( SimpleStatement.newInstance(statement).setKeyspace(this.keyspaceName))); } catch (Exception e) { - System.out.println(e.getMessage()); - System.out.println(e.fillInStackTrace()); - System.out.println(e); throw new IllegalArgumentException("Error reading collection.", e); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java index 1d122c6894..7a1939e0ac 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java @@ -26,7 +26,6 @@ import java.io.InputStream; import java.nio.charset.StandardCharsets; import java.time.format.DateTimeFormatter; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -121,9 +120,6 @@ public void createAndUploadCassandraConfigToGcs( String host = cassandraResourceManagers.getHost(); int port = cassandraResourceManagers.getPort(); String keyspaceName = cassandraResourceManagers.getKeyspaceName(); - System.out.println("Cassandra keyspaceName :: {}" + keyspaceName); - System.out.println("Cassandra host :: {}" + host); - System.out.println("Cassandra port :: {}" + port); String cassandraConfigContents; try (InputStream inputStream = Thread.currentThread() @@ -142,7 +138,6 @@ public void createAndUploadCassandraConfigToGcs( .replace("##port##", Integer.toString(port)) .replace("##keyspace##", keyspaceName); - System.out.println("Cassandra file contents: {}" + cassandraConfigContents); gcsResourceManager.createArtifact("input/cassandra-config.conf", cassandraConfigContents); } @@ -220,11 +215,8 @@ protected void createCassandraSchema( Resources.readLines( Resources.getResource(cassandraSchemaFile), StandardCharsets.UTF_8)); ddl = ddl.trim(); - System.out.println("DDL {}" + ddl); String[] ddls = ddl.split(";"); - System.out.println("DDLs statement {}" + Arrays.toString(ddls)); for (String d : ddls) { - System.out.println("DDL statement {}" + d); if (!d.isBlank()) { cassandraResourceManager.execute(d); } From 0422b9044e3e2a10961c477aabe49928b6958a1c Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 29 Jan 2025 09:53:44 +0530 Subject: [PATCH 06/19] Handle Catch --- .../teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java index 82a10d02de..7d584c09a7 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java @@ -341,7 +341,7 @@ private static Object handleSpannerColumnType( () -> { try { return convertBinaryEncodedStringToByteArray(hexEncodedString); - } catch (IllegalArgumentException e) { + } catch (Exception e) { return parseBlobType(hexEncodedString); } }); From 3c1a3ea2877005778a5dcd3874020556cb4c3543 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 29 Jan 2025 09:58:39 +0530 Subject: [PATCH 07/19] Handle with safehandler --- .../v2/templates/dbutils/dml/CassandraTypeHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java index 7d584c09a7..ec85050f42 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java @@ -340,8 +340,8 @@ private static Object handleSpannerColumnType( return safeHandle( () -> { try { - return convertBinaryEncodedStringToByteArray(hexEncodedString); - } catch (Exception e) { + return safeHandle(() -> convertBinaryEncodedStringToByteArray(hexEncodedString)); + } catch (IllegalArgumentException e) { return parseBlobType(hexEncodedString); } }); From a99cc0e3c471eef47f9e0f511350a98f1cc58cac Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 29 Jan 2025 15:26:44 +0530 Subject: [PATCH 08/19] Running load test --- .github/workflows/java-pr.yml | 184 +++++++++--------- .../v2/templates/SpannerToSourceDbLTBase.java | 3 +- 2 files changed, 94 insertions(+), 93 deletions(-) diff --git a/.github/workflows/java-pr.yml b/.github/workflows/java-pr.yml index a777e95712..21892e7b65 100644 --- a/.github/workflows/java-pr.yml +++ b/.github/workflows/java-pr.yml @@ -90,101 +90,101 @@ jobs: run: ./cicd/run-build - name: Cleanup Java Environment uses: ./.github/actions/cleanup-java-env - java_unit_tests: - name: Unit Tests - needs: [java_build] - timeout-minutes: 60 - runs-on: [self-hosted, it] - steps: - - name: Checkout Code - uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 - - name: Setup Environment - id: setup-env - uses: ./.github/actions/setup-env - - name: Run Unit Tests - run: ./cicd/run-unit-tests - - name: Upload Unit Tests Report - uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 - if: always() # always run even if the previous step fails - with: - name: surefire-test-results - path: '**/surefire-reports/TEST-*.xml' - retention-days: 1 - - name: Upload coverage reports to Codecov - uses: codecov/codecov-action@v4.0.1 - with: - token: ${{ secrets.CODECOV_TOKEN }} - slug: GoogleCloudPlatform/DataflowTemplates - files: 'target/site/jacoco-aggregate/jacoco.xml' - # Temp fix for https://github.com/codecov/codecov-action/issues/1487 - version: v0.6.0 - - name: Cleanup Java Environment - uses: ./.github/actions/cleanup-java-env - if: always() - java_integration_smoke_tests_templates: - name: Dataflow Templates Integration Smoke Tests - needs: [spotless_check, checkstyle_check, java_build, java_unit_tests] - timeout-minutes: 60 - # Run on any runner that matches all the specified runs-on values. - runs-on: [self-hosted, it] - steps: - - name: Checkout Code - uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 - - name: Setup Environment - id: setup-env - uses: ./.github/actions/setup-env - - name: Run Integration Smoke Tests - run: | - ./cicd/run-it-smoke-tests \ - --modules-to-build="DEFAULT" \ - --it-region="us-central1" \ - --it-project="cloud-teleport-testing" \ - --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ - --it-private-connectivity="datastream-private-connect-us-central1" - - name: Upload Smoke Tests Report - uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 - if: always() # always run even if the previous step fails - with: - name: surefire-test-results - path: '**/surefire-reports/TEST-*.xml' - retention-days: 1 - - name: Cleanup Java Environment - uses: ./.github/actions/cleanup-java-env - if: always() - java_integration_tests_templates: - name: Dataflow Templates Integration Tests - needs: [java_integration_smoke_tests_templates] - timeout-minutes: 240 - # Run on any runner that matches all the specified runs-on values. - runs-on: [self-hosted, it] - steps: - - name: Checkout Code - uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 - - name: Setup Environment - id: setup-env - uses: ./.github/actions/setup-env - - name: Run Integration Tests - run: | - ./cicd/run-it-tests \ - --modules-to-build="DEFAULT" \ - --it-region="us-central1" \ - --it-project="cloud-teleport-testing" \ - --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ - --it-private-connectivity="datastream-private-connect-us-central1" - - name: Upload Integration Tests Report - uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 - if: always() # always run even if the previous step fails - with: - name: surefire-test-results - path: '**/surefire-reports/TEST-*.xml' - retention-days: 1 - - name: Cleanup Java Environment - uses: ./.github/actions/cleanup-java-env - if: always() +# java_unit_tests: +# name: Unit Tests +# needs: [java_build] +# timeout-minutes: 60 +# runs-on: [self-hosted, it] +# steps: +# - name: Checkout Code +# uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 +# - name: Setup Environment +# id: setup-env +# uses: ./.github/actions/setup-env +# - name: Run Unit Tests +# run: ./cicd/run-unit-tests +# - name: Upload Unit Tests Report +# uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 +# if: always() # always run even if the previous step fails +# with: +# name: surefire-test-results +# path: '**/surefire-reports/TEST-*.xml' +# retention-days: 1 +# - name: Upload coverage reports to Codecov +# uses: codecov/codecov-action@v4.0.1 +# with: +# token: ${{ secrets.CODECOV_TOKEN }} +# slug: GoogleCloudPlatform/DataflowTemplates +# files: 'target/site/jacoco-aggregate/jacoco.xml' +# # Temp fix for https://github.com/codecov/codecov-action/issues/1487 +# version: v0.6.0 +# - name: Cleanup Java Environment +# uses: ./.github/actions/cleanup-java-env +# if: always() +# java_integration_smoke_tests_templates: +# name: Dataflow Templates Integration Smoke Tests +# needs: [spotless_check, checkstyle_check, java_build, java_unit_tests] +# timeout-minutes: 60 +# # Run on any runner that matches all the specified runs-on values. +# runs-on: [self-hosted, it] +# steps: +# - name: Checkout Code +# uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 +# - name: Setup Environment +# id: setup-env +# uses: ./.github/actions/setup-env +# - name: Run Integration Smoke Tests +# run: | +# ./cicd/run-it-smoke-tests \ +# --modules-to-build="DEFAULT" \ +# --it-region="us-central1" \ +# --it-project="cloud-teleport-testing" \ +# --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ +# --it-private-connectivity="datastream-private-connect-us-central1" +# - name: Upload Smoke Tests Report +# uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 +# if: always() # always run even if the previous step fails +# with: +# name: surefire-test-results +# path: '**/surefire-reports/TEST-*.xml' +# retention-days: 1 +# - name: Cleanup Java Environment +# uses: ./.github/actions/cleanup-java-env +# if: always() +# java_integration_tests_templates: +# name: Dataflow Templates Integration Tests +# needs: [java_integration_smoke_tests_templates] +# timeout-minutes: 240 +# # Run on any runner that matches all the specified runs-on values. +# runs-on: [self-hosted, it] +# steps: +# - name: Checkout Code +# uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 +# - name: Setup Environment +# id: setup-env +# uses: ./.github/actions/setup-env +# - name: Run Integration Tests +# run: | +# ./cicd/run-it-tests \ +# --modules-to-build="DEFAULT" \ +# --it-region="us-central1" \ +# --it-project="cloud-teleport-testing" \ +# --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ +# --it-private-connectivity="datastream-private-connect-us-central1" +# - name: Upload Integration Tests Report +# uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 +# if: always() # always run even if the previous step fails +# with: +# name: surefire-test-results +# path: '**/surefire-reports/TEST-*.xml' +# retention-days: 1 +# - name: Cleanup Java Environment +# uses: ./.github/actions/cleanup-java-env +# if: always() java_load_tests_templates: if: contains(github.event.pull_request.labels.*.name, 'run-load-tests') name: Dataflow Templates Load Tests - needs: [spotless_check, checkstyle_check, java_build, java_unit_tests, java_integration_tests_templates] + needs: [spotless_check, checkstyle_check, java_build] timeout-minutes: 600 # Run on any runner that matches all the specified runs-on values. runs-on: [self-hosted, perf] diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java index 43c0453adf..9aab705423 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java @@ -56,7 +56,8 @@ public class SpannerToSourceDbLTBase extends TemplateLoadTestBase { private static final String TEMPLATE_SPEC_PATH = MoreObjects.firstNonNull( - TestProperties.specPath(), "gs://dataflow-templates/latest/flex/Spanner_to_SourceDb"); + TestProperties.specPath(), + "gs://dataflow-templates-spanner-to-cassandra/templates/flex/Spanner_to_SourceDb"); public SpannerResourceManager spannerResourceManager; public SpannerResourceManager spannerMetadataResourceManager; public List jdbcResourceManagers; From 112d45f0a95bc55cf6afa1ac717d944a058921b0 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 29 Jan 2025 15:28:44 +0530 Subject: [PATCH 09/19] revert unwanted commit --- .github/workflows/java-pr.yml | 184 +++++++++--------- .../v2/templates/SpannerToSourceDbLTBase.java | 3 +- 2 files changed, 93 insertions(+), 94 deletions(-) diff --git a/.github/workflows/java-pr.yml b/.github/workflows/java-pr.yml index 21892e7b65..a777e95712 100644 --- a/.github/workflows/java-pr.yml +++ b/.github/workflows/java-pr.yml @@ -90,101 +90,101 @@ jobs: run: ./cicd/run-build - name: Cleanup Java Environment uses: ./.github/actions/cleanup-java-env -# java_unit_tests: -# name: Unit Tests -# needs: [java_build] -# timeout-minutes: 60 -# runs-on: [self-hosted, it] -# steps: -# - name: Checkout Code -# uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 -# - name: Setup Environment -# id: setup-env -# uses: ./.github/actions/setup-env -# - name: Run Unit Tests -# run: ./cicd/run-unit-tests -# - name: Upload Unit Tests Report -# uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 -# if: always() # always run even if the previous step fails -# with: -# name: surefire-test-results -# path: '**/surefire-reports/TEST-*.xml' -# retention-days: 1 -# - name: Upload coverage reports to Codecov -# uses: codecov/codecov-action@v4.0.1 -# with: -# token: ${{ secrets.CODECOV_TOKEN }} -# slug: GoogleCloudPlatform/DataflowTemplates -# files: 'target/site/jacoco-aggregate/jacoco.xml' -# # Temp fix for https://github.com/codecov/codecov-action/issues/1487 -# version: v0.6.0 -# - name: Cleanup Java Environment -# uses: ./.github/actions/cleanup-java-env -# if: always() -# java_integration_smoke_tests_templates: -# name: Dataflow Templates Integration Smoke Tests -# needs: [spotless_check, checkstyle_check, java_build, java_unit_tests] -# timeout-minutes: 60 -# # Run on any runner that matches all the specified runs-on values. -# runs-on: [self-hosted, it] -# steps: -# - name: Checkout Code -# uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 -# - name: Setup Environment -# id: setup-env -# uses: ./.github/actions/setup-env -# - name: Run Integration Smoke Tests -# run: | -# ./cicd/run-it-smoke-tests \ -# --modules-to-build="DEFAULT" \ -# --it-region="us-central1" \ -# --it-project="cloud-teleport-testing" \ -# --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ -# --it-private-connectivity="datastream-private-connect-us-central1" -# - name: Upload Smoke Tests Report -# uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 -# if: always() # always run even if the previous step fails -# with: -# name: surefire-test-results -# path: '**/surefire-reports/TEST-*.xml' -# retention-days: 1 -# - name: Cleanup Java Environment -# uses: ./.github/actions/cleanup-java-env -# if: always() -# java_integration_tests_templates: -# name: Dataflow Templates Integration Tests -# needs: [java_integration_smoke_tests_templates] -# timeout-minutes: 240 -# # Run on any runner that matches all the specified runs-on values. -# runs-on: [self-hosted, it] -# steps: -# - name: Checkout Code -# uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 -# - name: Setup Environment -# id: setup-env -# uses: ./.github/actions/setup-env -# - name: Run Integration Tests -# run: | -# ./cicd/run-it-tests \ -# --modules-to-build="DEFAULT" \ -# --it-region="us-central1" \ -# --it-project="cloud-teleport-testing" \ -# --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ -# --it-private-connectivity="datastream-private-connect-us-central1" -# - name: Upload Integration Tests Report -# uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 -# if: always() # always run even if the previous step fails -# with: -# name: surefire-test-results -# path: '**/surefire-reports/TEST-*.xml' -# retention-days: 1 -# - name: Cleanup Java Environment -# uses: ./.github/actions/cleanup-java-env -# if: always() + java_unit_tests: + name: Unit Tests + needs: [java_build] + timeout-minutes: 60 + runs-on: [self-hosted, it] + steps: + - name: Checkout Code + uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 + - name: Setup Environment + id: setup-env + uses: ./.github/actions/setup-env + - name: Run Unit Tests + run: ./cicd/run-unit-tests + - name: Upload Unit Tests Report + uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 + if: always() # always run even if the previous step fails + with: + name: surefire-test-results + path: '**/surefire-reports/TEST-*.xml' + retention-days: 1 + - name: Upload coverage reports to Codecov + uses: codecov/codecov-action@v4.0.1 + with: + token: ${{ secrets.CODECOV_TOKEN }} + slug: GoogleCloudPlatform/DataflowTemplates + files: 'target/site/jacoco-aggregate/jacoco.xml' + # Temp fix for https://github.com/codecov/codecov-action/issues/1487 + version: v0.6.0 + - name: Cleanup Java Environment + uses: ./.github/actions/cleanup-java-env + if: always() + java_integration_smoke_tests_templates: + name: Dataflow Templates Integration Smoke Tests + needs: [spotless_check, checkstyle_check, java_build, java_unit_tests] + timeout-minutes: 60 + # Run on any runner that matches all the specified runs-on values. + runs-on: [self-hosted, it] + steps: + - name: Checkout Code + uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 + - name: Setup Environment + id: setup-env + uses: ./.github/actions/setup-env + - name: Run Integration Smoke Tests + run: | + ./cicd/run-it-smoke-tests \ + --modules-to-build="DEFAULT" \ + --it-region="us-central1" \ + --it-project="cloud-teleport-testing" \ + --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ + --it-private-connectivity="datastream-private-connect-us-central1" + - name: Upload Smoke Tests Report + uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 + if: always() # always run even if the previous step fails + with: + name: surefire-test-results + path: '**/surefire-reports/TEST-*.xml' + retention-days: 1 + - name: Cleanup Java Environment + uses: ./.github/actions/cleanup-java-env + if: always() + java_integration_tests_templates: + name: Dataflow Templates Integration Tests + needs: [java_integration_smoke_tests_templates] + timeout-minutes: 240 + # Run on any runner that matches all the specified runs-on values. + runs-on: [self-hosted, it] + steps: + - name: Checkout Code + uses: actions/checkout@ee0669bd1cc54295c223e0bb666b733df41de1c5 # v2.7.0 + - name: Setup Environment + id: setup-env + uses: ./.github/actions/setup-env + - name: Run Integration Tests + run: | + ./cicd/run-it-tests \ + --modules-to-build="DEFAULT" \ + --it-region="us-central1" \ + --it-project="cloud-teleport-testing" \ + --it-artifact-bucket="cloud-teleport-testing-it-gitactions" \ + --it-private-connectivity="datastream-private-connect-us-central1" + - name: Upload Integration Tests Report + uses: actions/upload-artifact@0b7f8abb1508181956e8e162db84b466c27e18ce # v3.1.2 + if: always() # always run even if the previous step fails + with: + name: surefire-test-results + path: '**/surefire-reports/TEST-*.xml' + retention-days: 1 + - name: Cleanup Java Environment + uses: ./.github/actions/cleanup-java-env + if: always() java_load_tests_templates: if: contains(github.event.pull_request.labels.*.name, 'run-load-tests') name: Dataflow Templates Load Tests - needs: [spotless_check, checkstyle_check, java_build] + needs: [spotless_check, checkstyle_check, java_build, java_unit_tests, java_integration_tests_templates] timeout-minutes: 600 # Run on any runner that matches all the specified runs-on values. runs-on: [self-hosted, perf] diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java index 9aab705423..43c0453adf 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbLTBase.java @@ -56,8 +56,7 @@ public class SpannerToSourceDbLTBase extends TemplateLoadTestBase { private static final String TEMPLATE_SPEC_PATH = MoreObjects.firstNonNull( - TestProperties.specPath(), - "gs://dataflow-templates-spanner-to-cassandra/templates/flex/Spanner_to_SourceDb"); + TestProperties.specPath(), "gs://dataflow-templates/latest/flex/Spanner_to_SourceDb"); public SpannerResourceManager spannerResourceManager; public SpannerResourceManager spannerMetadataResourceManager; public List jdbcResourceManagers; From 4308686d274c5a91955d54a9457488f962507cda Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Thu, 30 Jan 2025 11:09:27 +0530 Subject: [PATCH 10/19] Added IT FIXES --- .../templates/SpannerToCassandraDbITBase.java | 40 +++++++++++++------ .../SpannerToCassandraSourceDbDatatypeIT.java | 5 ++- .../SpannerToSourceDbCassandraIT.java | 5 ++- ...e.conf => cassandra-config-template.conf} | 0 ...spanner-schema.sql => spanner-schema.sql} | 0 ...spanner-schema.sql => spanner-schema.sql} | 0 6 files changed, 35 insertions(+), 15 deletions(-) rename v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/{cassandra-config-template.conf => cassandra-config-template.conf} (100%) rename v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/{spanner-schema.sql => spanner-schema.sql} (100%) rename v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/{spanner-schema.sql => spanner-schema.sql} (100%) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java index 7a1939e0ac..64405b98b4 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java @@ -21,15 +21,18 @@ import com.google.common.io.Resources; import com.google.pubsub.v1.SubscriptionName; import com.google.pubsub.v1.TopicName; +import java.io.BufferedReader; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; import java.time.format.DateTimeFormatter; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.utils.PipelineUtils; import org.apache.beam.it.common.utils.ResourceManagerUtils; @@ -50,14 +53,27 @@ protected SpannerResourceManager createSpannerDatabase(String spannerSchemaFile) SpannerResourceManager.builder("rr-main-" + testName, PROJECT, REGION) .maybeUseStaticInstance() .build(); - String ddl = - String.join( - " ", - Resources.readLines(Resources.getResource(spannerSchemaFile), StandardCharsets.UTF_8)); - ddl = ddl.trim(); - String[] ddls = ddl.split(";"); + + String ddl; + try (InputStream inputStream = + Thread.currentThread().getContextClassLoader().getResourceAsStream(spannerSchemaFile)) { + if (inputStream == null) { + throw new FileNotFoundException("Resource file not found: " + spannerSchemaFile); + } + try (BufferedReader reader = + new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { + ddl = reader.lines().collect(Collectors.joining("\n")); + } + } + + if (ddl.isBlank()) { + throw new IllegalStateException("DDL file is empty: " + spannerSchemaFile); + } + + String[] ddls = ddl.trim().split(";"); for (String d : ddls) { - if (!d.isBlank()) { + d = d.trim(); + if (!d.isEmpty()) { spannerResourceManager.executeDdlStatement(d); } } @@ -114,7 +130,8 @@ public SubscriptionName createPubsubResources( public void createAndUploadCassandraConfigToGcs( GcsResourceManager gcsResourceManager, - CassandraSharedResourceManager cassandraResourceManagers) + CassandraSharedResourceManager cassandraResourceManagers, + String cassandraConfigFile) throws IOException { String host = cassandraResourceManagers.getHost(); @@ -122,12 +139,9 @@ public void createAndUploadCassandraConfigToGcs( String keyspaceName = cassandraResourceManagers.getKeyspaceName(); String cassandraConfigContents; try (InputStream inputStream = - Thread.currentThread() - .getContextClassLoader() - .getResourceAsStream("SpannerToCassandraSourceIT/cassandra-config-template.conf")) { + Thread.currentThread().getContextClassLoader().getResourceAsStream(cassandraConfigFile)) { if (inputStream == null) { - throw new FileNotFoundException( - "Resource file not found: SpannerToCassandraSourceIT/cassandra-config-template.conf"); + throw new FileNotFoundException("Resource file not found: " + cassandraConfigFile); } cassandraConfigContents = new String(inputStream.readAllBytes(), StandardCharsets.UTF_8); } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java index cad4b6550d..70cbe22ff5 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java @@ -64,6 +64,8 @@ public class SpannerToCassandraSourceDbDatatypeIT extends SpannerToCassandraDbIT "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; + private static final String CASSANDRA_CONFIG_FILE_RESOURCE = + "SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf"; private static final String TABLE = "AllDatatypeColumns"; private static final HashSet testInstances = @@ -96,7 +98,8 @@ public void setUp() throws IOException { GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) .build(); createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); - createAndUploadCassandraConfigToGcs(gcsResourceManager, cassandraResourceManager); + createAndUploadCassandraConfigToGcs( + gcsResourceManager, cassandraResourceManager, CASSANDRA_CONFIG_FILE_RESOURCE); pubsubResourceManager = setUpPubSubResourceManager(); subscriptionName = createPubsubResources( diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java index 67e09e173d..eb93ef5020 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java @@ -58,6 +58,8 @@ public class SpannerToSourceDbCassandraIT extends SpannerToCassandraDbITBase { "SpannerToCassandraSourceIT/spanner-schema.sql"; private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = "SpannerToCassandraSourceIT/cassandra-schema.sql"; + private static final String CASSANDRA_CONFIG_FILE_RESOURCE = + "SpannerToCassandraSourceIT/cassandra-config-template.conf"; private static final String TABLE = "Users"; private static final HashSet testInstances = new HashSet<>(); @@ -87,7 +89,8 @@ public void setUp() throws IOException { gcsResourceManager = GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) .build(); - createAndUploadCassandraConfigToGcs(gcsResourceManager, cassandraResourceManager); + createAndUploadCassandraConfigToGcs( + gcsResourceManager, cassandraResourceManager, CASSANDRA_CONFIG_FILE_RESOURCE); createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); pubsubResourceManager = setUpPubSubResourceManager(); subscriptionName = diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf similarity index 100% rename from v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf rename to v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql similarity index 100% rename from v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql rename to v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql similarity index 100% rename from v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql rename to v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql From 191c13c9208a99a61526f9eb12f0e7a7cf1354c7 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Thu, 30 Jan 2025 16:56:50 +0530 Subject: [PATCH 11/19] Added UT and removed unwanted SOUT --- .../dbutils/dml/CassandraTypeHandler.java | 10 ++---- .../CassandraSharedResourceManager.java | 6 +--- .../dbutils/dml/CassandraTypeHandlerTest.java | 34 +++++++++++++++++++ 3 files changed, 37 insertions(+), 13 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java index ec85050f42..ca4b5e6409 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java @@ -171,18 +171,12 @@ private static InetAddress handleCassandraInetAddressType(String inetString) { * @return a {@link ByteBuffer} object containing the value represented in cassandra type. */ private static ByteBuffer parseBlobType(Object colValue) { - byte[] byteArray; - if (colValue instanceof byte[]) { - byteArray = (byte[]) colValue; - } else if (colValue instanceof String) { - byteArray = java.util.Base64.getDecoder().decode((String) colValue); + return ByteBuffer.wrap((byte[]) colValue); } else if (colValue instanceof ByteBuffer) { return (ByteBuffer) colValue; - } else { - throw new IllegalArgumentException("Unsupported type for column"); } - return ByteBuffer.wrap(byteArray); + return ByteBuffer.wrap(java.util.Base64.getDecoder().decode((String) colValue)); } /** diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java index 27c200d18d..5f166a2317 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java @@ -72,7 +72,7 @@ public class CassandraSharedResourceManager private CassandraSharedResourceManager(Builder builder) { this( - /* cassandraClient= */ null, + null, new CassandraContainer<>( DockerImageName.parse(builder.containerImageName).withTag(builder.containerImageTag)), builder); @@ -99,7 +99,6 @@ private CassandraSharedResourceManager(Builder builder) { : cassandraClient; if (!usingStaticDatabase) { - // Keyspace request may timeout on a few environments, if Cassandra is warming up Failsafe.with(buildRetryPolicy()) .run( () -> @@ -173,9 +172,6 @@ public synchronized void execute(String statement) { cassandraClient.execute( SimpleStatement.newInstance(statement).setKeyspace(this.keyspaceName))); } catch (Exception e) { - System.out.println(e.getMessage()); - System.out.println(e.fillInStackTrace()); - System.out.println(e); throw new IllegalArgumentException("Error reading collection.", e); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java index f6fcbc1812..8140647351 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandlerTest.java @@ -251,6 +251,32 @@ public void testGetColumnValueByTypeForStringHex() { assertArrayEquals(expectedBytes, actualBytes); } + @Test + public void testColumnKeyNotPresent() { + SpannerColumnType spannerType = new SpannerColumnType("bytes", false); + SourceColumnType sourceColumnType = new SourceColumnType("blob", null, null); + String columnName = "lastName"; + byte[] expectedBytes = new byte[] {1, 2, 3, 4, 5}; + StringBuilder binaryString = new StringBuilder(); + for (byte b : expectedBytes) { + binaryString.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0')); + } + String columnValue = binaryString.toString(); + String sourceDbTimezoneOffset = null; + + SpannerColumnDefinition spannerColDef = new SpannerColumnDefinition(columnName, spannerType); + SourceColumnDefinition sourceColDef = new SourceColumnDefinition(columnName, sourceColumnType); + + JSONObject valuesJson = new JSONObject(); + valuesJson.put("random", columnValue); + + PreparedStatementValueObject result = + getColumnValueByType(spannerColDef, sourceColDef, valuesJson, sourceDbTimezoneOffset); + + assertEquals("blob", result.dataType()); + assertEquals(CassandraTypeHandler.NullClass.INSTANCE, result.value()); + } + @Test public void testGetColumnValueByTypeForStringByteArrayBase64Encode() { SpannerColumnType spannerType = new SpannerColumnType("bytes", false); @@ -1257,6 +1283,14 @@ public void testHandleCassandraVarintType_ForBytesArray() { assertEquals(expected, result); } + @Test + public void testHandleCassandraVarintType_ForByteBuffer() { + byte[] byteArray = new byte[] {0, 0, 0, 0, 0, 0, 0, 10}; + BigInteger expected = new BigInteger(byteArray); + Object result = CassandraTypeHandler.castToExpectedType("varint", ByteBuffer.wrap(byteArray)); + assertEquals(expected, result); + } + @Test public void testHandleCassandraVarintType_ForInteger() { Long inputValue = 123456789L; From 7d288f01ef4cebd5f483c6be2ccaa9ef9c38c4dd Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 5 Feb 2025 11:26:41 +0530 Subject: [PATCH 12/19] Added One to Many Datatype Transformation for IT (#90) Added One to Many Datatype Transformation for IT Fix PR review comments --- ...ger.java => CassandraResourceManager.java} | 32 +- .../templates/SpannerToCassandraDbITBase.java | 245 ----- .../SpannerToCassandraSourceDbDatatypeIT.java | 422 -------- .../SpannerToCassandraSourceDbIT.java | 928 ++++++++++++++++++ .../SpannerToSourceDbCassandraIT.java | 226 ----- .../SpannerToSourceDbCustomShardIT.java | 1 + ...annerToSourceDbCustomTransformationIT.java | 3 +- .../SpannerToSourceDbDatatypeIT.java | 1 + .../v2/templates/SpannerToSourceDbIT.java | 1 + .../v2/templates/SpannerToSourceDbITBase.java | 108 +- ...annerToSourceDbInterleaveMultiShardIT.java | 1 + .../SpannerToSourceDbTimezoneIT.java | 1 + .../cassandra-config-template.conf | 12 - .../cassandra-schema.sql | 43 - .../spanner-schema.sql | 49 - .../cassandra-schema.sql | 88 ++ .../spanner-schema.sql | 88 ++ 17 files changed, 1218 insertions(+), 1031 deletions(-) rename v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/{CassandraSharedResourceManager.java => CassandraResourceManager.java} (91%) delete mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java delete mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java create mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java delete mode 100644 v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java delete mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf delete mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql delete mode 100644 v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java similarity index 91% rename from v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java rename to v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java index 5f166a2317..ab1dbf39c8 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraSharedResourceManager.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java @@ -33,7 +33,6 @@ import org.apache.beam.it.common.utils.ResourceManagerUtils; import org.apache.beam.it.testcontainers.TestContainerResourceManager; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,10 +51,10 @@ * *

The class is thread-safe. */ -public class CassandraSharedResourceManager - extends TestContainerResourceManager> implements ResourceManager { +public class CassandraResourceManager extends TestContainerResourceManager> + implements ResourceManager { - private static final Logger LOG = LoggerFactory.getLogger(CassandraSharedResourceManager.class); + private static final Logger LOG = LoggerFactory.getLogger(CassandraResourceManager.class); private static final String DEFAULT_CASSANDRA_CONTAINER_NAME = "cassandra"; @@ -70,7 +69,7 @@ public class CassandraSharedResourceManager private final String keyspaceName; private final boolean usingStaticDatabase; - private CassandraSharedResourceManager(Builder builder) { + private CassandraResourceManager(Builder builder) { this( null, new CassandraContainer<>( @@ -80,7 +79,7 @@ private CassandraSharedResourceManager(Builder builder) { @VisibleForTesting @SuppressWarnings("nullness") - CassandraSharedResourceManager( + CassandraResourceManager( @Nullable CqlSession cassandraClient, CassandraContainer container, Builder builder) { super(container, builder); // we are trying to handle userDefined KeyspaceName name without usingStatic Container @@ -176,19 +175,6 @@ public synchronized void execute(String statement) { } } - /** - * Inserts the given Document into a table. - * - *

A database will be created here, if one does not already exist. - * - * @param tableName The name of the table to insert the document into. - * @param document The document to insert into the table. - * @return A boolean indicating whether the Document was inserted successfully. - */ - public synchronized boolean insertDocument(String tableName, Map document) { - return insertDocuments(tableName, ImmutableList.of(document)); - } - /** * Inserts the given Documents into a collection. * @@ -312,9 +298,9 @@ private static RetryPolicy buildRetryPolicy() { .build(); } - /** Builder for {@link CassandraSharedResourceManager}. */ + /** Builder for {@link CassandraResourceManager}. */ public static final class Builder - extends TestContainerResourceManager.Builder { + extends TestContainerResourceManager.Builder { private @Nullable String keyspaceName; @@ -359,8 +345,8 @@ public Builder sePreGeneratedKeyspaceName(boolean preGeneratedKeyspaceName) { } @Override - public CassandraSharedResourceManager build() { - return new CassandraSharedResourceManager(this); + public CassandraResourceManager build() { + return new CassandraResourceManager(this); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java deleted file mode 100644 index 64405b98b4..0000000000 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraDbITBase.java +++ /dev/null @@ -1,245 +0,0 @@ -/* - * Copyright (C) 2025 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package com.google.cloud.teleport.v2.templates; - -import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; - -import com.google.cloud.teleport.v2.spanner.migrations.transformation.CustomTransformation; -import com.google.common.io.Resources; -import com.google.pubsub.v1.SubscriptionName; -import com.google.pubsub.v1.TopicName; -import java.io.BufferedReader; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.time.format.DateTimeFormatter; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import org.apache.beam.it.common.PipelineLauncher; -import org.apache.beam.it.common.utils.PipelineUtils; -import org.apache.beam.it.common.utils.ResourceManagerUtils; -import org.apache.beam.it.gcp.TemplateTestBase; -import org.apache.beam.it.gcp.artifacts.utils.ArtifactUtils; -import org.apache.beam.it.gcp.pubsub.PubsubResourceManager; -import org.apache.beam.it.gcp.spanner.SpannerResourceManager; -import org.apache.beam.it.gcp.storage.GcsResourceManager; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public abstract class SpannerToCassandraDbITBase extends TemplateTestBase { - private static final Logger LOG = LoggerFactory.getLogger(SpannerToCassandraDbITBase.class); - - protected SpannerResourceManager createSpannerDatabase(String spannerSchemaFile) - throws IOException { - SpannerResourceManager spannerResourceManager = - SpannerResourceManager.builder("rr-main-" + testName, PROJECT, REGION) - .maybeUseStaticInstance() - .build(); - - String ddl; - try (InputStream inputStream = - Thread.currentThread().getContextClassLoader().getResourceAsStream(spannerSchemaFile)) { - if (inputStream == null) { - throw new FileNotFoundException("Resource file not found: " + spannerSchemaFile); - } - try (BufferedReader reader = - new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { - ddl = reader.lines().collect(Collectors.joining("\n")); - } - } - - if (ddl.isBlank()) { - throw new IllegalStateException("DDL file is empty: " + spannerSchemaFile); - } - - String[] ddls = ddl.trim().split(";"); - for (String d : ddls) { - d = d.trim(); - if (!d.isEmpty()) { - spannerResourceManager.executeDdlStatement(d); - } - } - return spannerResourceManager; - } - - protected SpannerResourceManager createSpannerMetadataDatabase() throws IOException { - SpannerResourceManager spannerMetadataResourceManager = - SpannerResourceManager.builder("rr-meta-" + testName, PROJECT, REGION) - .maybeUseStaticInstance() - .build(); - String dummy = "create table t1(id INT64 ) primary key(id)"; - spannerMetadataResourceManager.executeDdlStatement(dummy); - return spannerMetadataResourceManager; - } - - public PubsubResourceManager setUpPubSubResourceManager() throws IOException { - return PubsubResourceManager.builder(testName, PROJECT, credentialsProvider).build(); - } - - public CassandraSharedResourceManager generateKeyspaceAndBuildCassandraResource() { - String keyspaceName = - ResourceManagerUtils.generateResourceId( - testName, - Pattern.compile("[/\\\\. \"\u0000$]"), - "-", - 27, - DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss-SSSSSS")) - .replace('-', '_'); - if (keyspaceName.length() > 48) { - keyspaceName = keyspaceName.substring(0, 48); - } - return CassandraSharedResourceManager.builder(testName) - .setKeyspaceName(keyspaceName) - .sePreGeneratedKeyspaceName(true) - .build(); - } - - public SubscriptionName createPubsubResources( - String identifierSuffix, PubsubResourceManager pubsubResourceManager, String gcsPrefix) { - String topicNameSuffix = "rr-it" + identifierSuffix; - String subscriptionNameSuffix = "rr-it-sub" + identifierSuffix; - TopicName topic = pubsubResourceManager.createTopic(topicNameSuffix); - SubscriptionName subscription = - pubsubResourceManager.createSubscription(topic, subscriptionNameSuffix); - String prefix = gcsPrefix; - if (prefix.startsWith("/")) { - prefix = prefix.substring(1); - } - prefix += "/retry/"; - gcsClient.createNotification(topic.toString(), prefix); - return subscription; - } - - public void createAndUploadCassandraConfigToGcs( - GcsResourceManager gcsResourceManager, - CassandraSharedResourceManager cassandraResourceManagers, - String cassandraConfigFile) - throws IOException { - - String host = cassandraResourceManagers.getHost(); - int port = cassandraResourceManagers.getPort(); - String keyspaceName = cassandraResourceManagers.getKeyspaceName(); - String cassandraConfigContents; - try (InputStream inputStream = - Thread.currentThread().getContextClassLoader().getResourceAsStream(cassandraConfigFile)) { - if (inputStream == null) { - throw new FileNotFoundException("Resource file not found: " + cassandraConfigFile); - } - cassandraConfigContents = new String(inputStream.readAllBytes(), StandardCharsets.UTF_8); - } - - cassandraConfigContents = - cassandraConfigContents - .replace("##host##", host) - .replace("##port##", Integer.toString(port)) - .replace("##keyspace##", keyspaceName); - - gcsResourceManager.createArtifact("input/cassandra-config.conf", cassandraConfigContents); - } - - public PipelineLauncher.LaunchInfo launchDataflowJob( - GcsResourceManager gcsResourceManager, - SpannerResourceManager spannerResourceManager, - SpannerResourceManager spannerMetadataResourceManager, - String subscriptionName, - String identifierSuffix, - String shardingCustomJarPath, - String shardingCustomClassName, - String sourceDbTimezoneOffset, - CustomTransformation customTransformation) - throws IOException { - - Map params = - new HashMap<>() { - { - put("instanceId", spannerResourceManager.getInstanceId()); - put("databaseId", spannerResourceManager.getDatabaseId()); - put("spannerProjectId", PROJECT); - put("metadataDatabase", spannerMetadataResourceManager.getDatabaseId()); - put("metadataInstance", spannerMetadataResourceManager.getInstanceId()); - put( - "sourceShardsFilePath", - getGcsPath("input/cassandra-config.conf", gcsResourceManager)); - put("changeStreamName", "allstream"); - put("dlqGcsPubSubSubscription", subscriptionName); - put("deadLetterQueueDirectory", getGcsPath("dlq", gcsResourceManager)); - put("maxShardConnections", "5"); - put("maxNumWorkers", "1"); - put("numWorkers", "1"); - put("sourceType", "cassandra"); - } - }; - - if (shardingCustomJarPath != null) { - params.put( - "shardingCustomJarPath", - getGcsFullPath(gcsResourceManager, shardingCustomJarPath, identifierSuffix)); - } - if (shardingCustomClassName != null) { - params.put("shardingCustomClassName", shardingCustomClassName); - } - - if (sourceDbTimezoneOffset != null) { - params.put("sourceDbTimezoneOffset", sourceDbTimezoneOffset); - } - - if (customTransformation != null) { - params.put( - "transformationJarPath", getGcsPath(customTransformation.jarPath(), gcsResourceManager)); - params.put("transformationClassName", customTransformation.classPath()); - } - - // Construct template - String jobName = PipelineUtils.createJobName("rrev-it" + testName); - // /-DunifiedWorker=true when using runner v2 - PipelineLauncher.LaunchConfig.Builder options = - PipelineLauncher.LaunchConfig.builder(jobName, specPath); - options.setParameters(params); - options.addEnvironment("additionalExperiments", Collections.singletonList("use_runner_v2")); - // Run - PipelineLauncher.LaunchInfo jobInfo = launchTemplate(options, false); - assertThatPipeline(jobInfo).isRunning(); - return jobInfo; - } - - protected void createCassandraSchema( - CassandraSharedResourceManager cassandraResourceManager, String cassandraSchemaFile) - throws IOException { - String ddl = - String.join( - " ", - Resources.readLines( - Resources.getResource(cassandraSchemaFile), StandardCharsets.UTF_8)); - ddl = ddl.trim(); - String[] ddls = ddl.split(";"); - for (String d : ddls) { - if (!d.isBlank()) { - cassandraResourceManager.execute(d); - } - } - } - - public String getGcsFullPath( - GcsResourceManager gcsResourceManager, String artifactId, String identifierSuffix) { - return ArtifactUtils.getFullGcsPath( - artifactBucketName, identifierSuffix, gcsResourceManager.runId(), artifactId); - } -} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java deleted file mode 100644 index 70cbe22ff5..0000000000 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbDatatypeIT.java +++ /dev/null @@ -1,422 +0,0 @@ -/* - * Copyright (C) 2025 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package com.google.cloud.teleport.v2.templates; - -import static com.google.common.truth.Truth.assertThat; -import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; -import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; - -import com.datastax.oss.driver.api.core.cql.ResultSet; -import com.datastax.oss.driver.api.core.cql.Row; -import com.google.cloud.ByteArray; -import com.google.cloud.Date; -import com.google.cloud.Timestamp; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Value; -import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; -import com.google.cloud.teleport.metadata.TemplateIntegrationTest; -import com.google.pubsub.v1.SubscriptionName; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -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.pubsub.PubsubResourceManager; -import org.apache.beam.it.gcp.spanner.SpannerResourceManager; -import org.apache.beam.it.gcp.storage.GcsResourceManager; -import org.apache.commons.codec.DecoderException; -import org.apache.commons.codec.binary.Hex; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.junit.runners.model.MultipleFailureException; - -@Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) -@TemplateIntegrationTest(SpannerToSourceDb.class) -@RunWith(JUnit4.class) -public class SpannerToCassandraSourceDbDatatypeIT extends SpannerToCassandraDbITBase { - - private static final String SPANNER_DDL_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql"; - private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql"; - private static final String CASSANDRA_CONFIG_FILE_RESOURCE = - "SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf"; - - private static final String TABLE = "AllDatatypeColumns"; - private static final HashSet testInstances = - new HashSet<>(); - private static PipelineLauncher.LaunchInfo jobInfo; - public static SpannerResourceManager spannerResourceManager; - private static SpannerResourceManager spannerMetadataResourceManager; - public static CassandraSharedResourceManager cassandraResourceManager; - private static GcsResourceManager gcsResourceManager; - private static PubsubResourceManager pubsubResourceManager; - private SubscriptionName subscriptionName; - private final List assertionErrors = new ArrayList<>(); - - /** - * Setup resource managers and Launch dataflow job once during the execution of this test class. - * - * @throws IOException - */ - @Before - public void setUp() throws IOException { - skipBaseCleanup = true; - synchronized (SpannerToCassandraSourceDbDatatypeIT.class) { - testInstances.add(this); - if (jobInfo == null) { - spannerResourceManager = createSpannerDatabase(SPANNER_DDL_RESOURCE); - spannerMetadataResourceManager = createSpannerMetadataDatabase(); - - cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); - gcsResourceManager = - GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) - .build(); - createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); - createAndUploadCassandraConfigToGcs( - gcsResourceManager, cassandraResourceManager, CASSANDRA_CONFIG_FILE_RESOURCE); - pubsubResourceManager = setUpPubSubResourceManager(); - subscriptionName = - createPubsubResources( - getClass().getSimpleName(), - pubsubResourceManager, - getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); - jobInfo = - launchDataflowJob( - gcsResourceManager, - spannerResourceManager, - spannerMetadataResourceManager, - subscriptionName.toString(), - null, - null, - null, - null, - null); - } - } - } - - /** - * Cleanup dataflow job and all the resources and resource managers. - * - * @throws IOException - */ - @AfterClass - public static void cleanUp() throws IOException { - for (SpannerToCassandraSourceDbDatatypeIT instance : testInstances) { - instance.tearDownBase(); - } - ResourceManagerUtils.cleanResources( - spannerResourceManager, - cassandraResourceManager, - spannerMetadataResourceManager, - gcsResourceManager, - pubsubResourceManager); - } - - @Test - public void spannerToCassandraSourceDataTypeConversionTest() - throws InterruptedException, IOException, MultipleFailureException { - assertThatPipeline(jobInfo).isRunning(); - writeRowInSpanner(); - assertRowInCassandraDB(); - } - - private long getRowCount() { - String query = String.format("SELECT COUNT(*) FROM %s", TABLE); - ResultSet resultSet = cassandraResourceManager.executeStatement(query); - Row row = resultSet.one(); - if (row != null) { - return row.getLong(0); - } else { - throw new RuntimeException("Query did not return a result for table: " + TABLE); - } - } - - private void writeRowInSpanner() { - Mutation mutation = - Mutation.newInsertOrUpdateBuilder(TABLE) - .set("varchar_column") - .to("SampleVarchar") - .set("tinyint_column") - .to(127) - .set("text_column") - .to("This is some sample text data for the text column.") - .set("date_column") - .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) - .set("smallint_column") - .to(32767) - .set("mediumint_column") - .to(8388607) - .set("int_column") - .to(2147483647) - .set("bigint_column") - .to(9223372036854775807L) - .set("float_column") - .to(3.14159) - .set("double_column") - .to(2.718281828459045) - .set("decimal_column") - .to(new BigDecimal("12345.6789")) - .set("datetime_column") - .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) - .set("timestamp_column") - .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) - .set("time_column") - .to("12:30:00") - .set("year_column") - .to("2025") - .set("char_column") - .to("CHAR_DATA") - .set("tinytext_column") - .to("Short text for tinytext.") - .set("mediumtext_column") - .to("Longer text data for mediumtext column.") - .set("longtext_column") - .to("Very long text data that exceeds the medium text column length for long text.") - .set("enum_column") - .to("OptionA") - .set("bool_column") - .to(Value.bool(Boolean.TRUE)) - .set("other_bool_column") - .to(Value.bool(Boolean.FALSE)) - .set("bytes_column") - .to(Value.bytes(ByteArray.copyFrom("SGVsbG8gd29ybGQ=".getBytes()))) - .set("list_text_column") - .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) - .set("list_int_column") - .to(Value.json("[1, 2, 3, 4, 5]")) - .set("frozen_list_bigint_column") - .to(Value.json("[123456789012345, 987654321012345]")) - .set("set_text_column") - .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) - .set("set_date_column") - .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) - .set("frozen_set_bool_column") - .to(Value.json("[true, false]")) - .set("map_text_to_int_column") - .to(Value.json("{\"key1\": 10, \"key2\": 20}")) - .set("map_date_to_text_column") - .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) - .set("frozen_map_int_to_bool_column") - .to(Value.json("{\"1\": true, \"2\": false}")) - .set("map_text_to_list_column") - .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) - .set("map_text_to_set_column") - .to( - Value.json( - "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) - .set("set_of_maps_column") - .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) - .set("list_of_sets_column") - .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) - .set("frozen_map_text_to_list_column") - .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) - .set("frozen_map_text_to_set_column") - .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) - .set("frozen_set_of_maps_column") - .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) - .set("frozen_list_of_sets_column") - .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) - .set("varint_column") - .to(Value.bytes(ByteArray.copyFrom("b3f5ed4f".getBytes()))) - .build(); - - spannerResourceManager.write(mutation); - } - - private void assertAll(Runnable... assertions) throws MultipleFailureException { - for (Runnable assertion : assertions) { - try { - assertion.run(); - } catch (AssertionError e) { - assertionErrors.add(e); - } - } - if (!assertionErrors.isEmpty()) { - throw new MultipleFailureException(assertionErrors); - } - } - - private void assertRowInCassandraDB() throws InterruptedException, MultipleFailureException { - PipelineOperator.Result result = - pipelineOperator() - .waitForCondition( - createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); - assertThatResult(result).meetsConditions(); - Iterable rows; - try { - rows = cassandraResourceManager.readTable(TABLE); - } catch (Exception e) { - throw new RuntimeException("Failed to read from Cassandra table: " + TABLE, e); - } - - assertThat(rows).hasSize(1); - - Row row = rows.iterator().next(); - - assertThat(rows).hasSize(1); - assertAll( - // Basic Data Types - () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), - () -> assertThat(row.getLong("bigint_column")).isEqualTo(9223372036854775807L), - () -> assertThat(row.getBoolean("bool_column")).isTrue(), - () -> { - String hexString = "5347567362473867643239796247513d"; - byte[] byteArray; - try { - byteArray = Hex.decodeHex(hexString); - } catch (DecoderException e) { - byteArray = new byte[0]; - } - ByteBuffer expectedBuffer = ByteBuffer.wrap(byteArray); - assertThat(row.getByteBuffer("bytes_column")).isEqualTo(expectedBuffer); - }, - () -> assertThat(row.getString("char_column")).isEqualTo("CHAR_DATA"), - () -> - assertThat(row.getLocalDate("date_column")) - .isEqualTo(java.time.LocalDate.of(2025, 1, 27)), - () -> - assertThat(row.getInstant("datetime_column")) - .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), - () -> - assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("12345.6789")), - () -> assertThat(row.getDouble("double_column")).isEqualTo(2.718281828459045), - () -> assertThat(row.getFloat("float_column")).isEqualTo(3.14159f), - - // Collections (frozen, list, set, map) - () -> - assertThat(row.getList("frozen_list_bigint_column", Long.class)) - .isEqualTo(Arrays.asList(123456789012345L, 987654321012345L)), - () -> - assertThat(row.getSet("frozen_set_bool_column", Boolean.class)) - .isEqualTo(new HashSet<>(Arrays.asList(false, true))), - () -> - assertThat(row.getMap("frozen_map_int_to_bool_column", Integer.class, Boolean.class)) - .isEqualTo(Map.of(1, true, 2, false)), - () -> - assertThat(row.getMap("frozen_map_text_to_list_column", String.class, List.class)) - .isEqualTo(Map.of("fruits", Arrays.asList("apple", "banana"))), - () -> - assertThat(row.getMap("frozen_map_text_to_set_column", String.class, Set.class)) - .isEqualTo(Map.of("vegetables", new HashSet<>(Arrays.asList("carrot", "spinach")))), - () -> - assertThat(row.getSet("frozen_set_of_maps_column", Map.class)) - .isEqualTo( - new HashSet<>( - Arrays.asList( - Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), - - // Lists and Sets - () -> - assertThat(row.getList("list_int_column", Integer.class)) - .isEqualTo(Arrays.asList(1, 2, 3, 4, 5)), - () -> - assertThat(row.getList("list_text_column", String.class)) - .isEqualTo(Arrays.asList("apple", "banana", "cherry")), - () -> - assertThat(row.getList("list_of_sets_column", Set.class)) - .isEqualTo( - Arrays.asList( - new HashSet<>(Arrays.asList("apple", "banana")), - new HashSet<>(Arrays.asList("carrot", "spinach")))), - - // Maps - () -> - assertThat( - row.getMap("map_date_to_text_column", java.time.LocalDate.class, String.class)) - .isEqualTo( - Map.of( - java.time.LocalDate.parse("2025-01-27"), "event1", - java.time.LocalDate.parse("2025-02-01"), "event2")), - () -> - assertThat(row.getMap("map_text_to_int_column", String.class, Integer.class)) - .isEqualTo(Map.of("key1", 10, "key2", 20)), - () -> - assertThat(row.getMap("map_text_to_list_column", String.class, List.class)) - .isEqualTo( - Map.of( - "color", - Arrays.asList("red", "green"), - "fruit", - Arrays.asList("apple", "banana"))), - () -> - assertThat(row.getMap("map_text_to_set_column", String.class, Set.class)) - .isEqualTo( - Map.of( - "fruit", - new HashSet<>(Arrays.asList("apple", "banana")), - "vegetables", - new HashSet<>(Arrays.asList("carrot", "spinach")))), - - // Sets - () -> - assertThat(row.getSet("set_date_column", java.time.LocalDate.class)) - .isEqualTo( - new HashSet<>( - Arrays.asList( - java.time.LocalDate.parse("2025-01-27"), - java.time.LocalDate.parse("2025-02-01")))), - () -> - assertThat(row.getSet("set_text_column", String.class)) - .isEqualTo(new HashSet<>(Arrays.asList("apple", "orange", "banana"))), - () -> - assertThat(row.getSet("set_of_maps_column", Map.class)) - .isEqualTo( - new HashSet<>( - Arrays.asList( - Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), - - // Other Basic Types - () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 32767), - () -> assertThat(row.getInt("mediumint_column")).isEqualTo(8388607), - () -> assertThat(row.getInt("int_column")).isEqualTo(2147483647), - () -> assertThat(row.getString("enum_column")).isEqualTo("OptionA"), - () -> assertThat(row.getString("year_column")).isEqualTo("2025"), - () -> - assertThat(row.getString("longtext_column")) - .isEqualTo( - "Very long text data that exceeds the medium text column length for long text."), - () -> assertThat(row.getString("tinytext_column")).isEqualTo("Short text for tinytext."), - () -> - assertThat(row.getString("mediumtext_column")) - .isEqualTo("Longer text data for mediumtext column."), - () -> - assertThat(row.getString("text_column")) - .isEqualTo("This is some sample text data for the text column."), - () -> - assertThat(row.getLocalTime("time_column")) - .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), - () -> - assertThat(row.getInstant("timestamp_column")) - .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), - () -> - assertThat(row.getBigInteger("varint_column")) - .isEqualTo(java.math.BigInteger.valueOf(7076111819049546854L))); - } -} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java new file mode 100644 index 0000000000..f16c24c0c0 --- /dev/null +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java @@ -0,0 +1,928 @@ +/* + * Copyright (C) 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +package com.google.cloud.teleport.v2.templates; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; +import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; + +import com.datastax.oss.driver.api.core.cql.ResultSet; +import com.datastax.oss.driver.api.core.cql.Row; +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Options; +import com.google.cloud.spanner.TransactionRunner; +import com.google.cloud.spanner.Value; +import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; +import com.google.cloud.teleport.metadata.TemplateIntegrationTest; +import com.google.pubsub.v1.SubscriptionName; +import java.io.IOException; +import java.math.BigDecimal; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +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.pubsub.PubsubResourceManager; +import org.apache.beam.it.gcp.spanner.SpannerResourceManager; +import org.apache.beam.it.gcp.storage.GcsResourceManager; +import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; +import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.junit.runners.model.MultipleFailureException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) +@TemplateIntegrationTest(SpannerToSourceDb.class) +@RunWith(JUnit4.class) +public class SpannerToCassandraSourceDbIT extends SpannerToSourceDbITBase { + + private static final Logger LOG = LoggerFactory.getLogger(SpannerToCassandraSourceDbIT.class); + + private static final String SPANNER_DDL_RESOURCE = + "SpannerToCassandraSourceIT/spanner-schema.sql"; + private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = + "SpannerToCassandraSourceIT/cassandra-schema.sql"; + private static final String CASSANDRA_CONFIG_FILE_RESOURCE = + "SpannerToCassandraSourceIT/cassandra-config-template.conf"; + + private static final String USER_TABLE = "Users"; + private static final String ALL_DATA_TYPES_TABLE = "AllDatatypeColumns"; + private static final String ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE = "AllDatatypeTransformation"; + private static final HashSet testInstances = new HashSet<>(); + private static PipelineLauncher.LaunchInfo jobInfo; + public static SpannerResourceManager spannerResourceManager; + private static SpannerResourceManager spannerMetadataResourceManager; + public static CassandraResourceManager cassandraResourceManager; + private static GcsResourceManager gcsResourceManager; + private static PubsubResourceManager pubsubResourceManager; + private SubscriptionName subscriptionName; + private final List assertionErrors = new ArrayList<>(); + + /** + * Setup resource managers and Launch dataflow job once during the execution of this test class. + * + * @throws IOException + */ + @Before + public void setUp() throws IOException { + skipBaseCleanup = true; + synchronized (SpannerToCassandraSourceDbIT.class) { + testInstances.add(this); + if (jobInfo == null) { + spannerResourceManager = createSpannerDatabase(SPANNER_DDL_RESOURCE); + spannerMetadataResourceManager = createSpannerMetadataDatabase(); + + cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); + gcsResourceManager = + GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) + .build(); + createAndUploadCassandraConfigToGcs( + gcsResourceManager, cassandraResourceManager, CASSANDRA_CONFIG_FILE_RESOURCE); + createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); + pubsubResourceManager = setUpPubSubResourceManager(); + subscriptionName = + createPubsubResources( + getClass().getSimpleName(), + pubsubResourceManager, + getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); + jobInfo = + launchDataflowJob( + gcsResourceManager, + spannerResourceManager, + spannerMetadataResourceManager, + subscriptionName.toString(), + null, + null, + null, + null, + null, + "cassandra"); + } + } + } + + /** + * Cleanup dataflow job and all the resources and resource managers. + * + * @throws IOException + */ + @AfterClass + public static void cleanUp() throws IOException { + for (SpannerToCassandraSourceDbIT instance : testInstances) { + instance.tearDownBase(); + } + ResourceManagerUtils.cleanResources( + spannerResourceManager, + cassandraResourceManager, + spannerMetadataResourceManager, + gcsResourceManager, + pubsubResourceManager); + } + + /** + * Tests the data flow from Spanner to Cassandra. + * + *

This test ensures that a basic row is successfully written to Spanner and subsequently + * appears in Cassandra, validating end-to-end data consistency. + * + * @throws InterruptedException if the thread is interrupted during execution. + * @throws IOException if an I/O error occurs during the test execution. + */ + @Test + public void spannerToCasandraSourceDbBasic() throws InterruptedException, IOException { + assertThatPipeline(jobInfo).isRunning(); + writeBasicRowInSpanner(); + assertBasicRowInCassandraDB(); + } + + /** + * Tests the data type conversion from Spanner to Cassandra. + * + *

This test ensures that all supported data types are correctly written to Spanner and + * subsequently retrieved from Cassandra, verifying data integrity and type conversions. + * + * @throws InterruptedException if the thread is interrupted during execution. + * @throws IOException if an I/O error occurs during the test execution. + * @throws MultipleFailureException if multiple assertions fail during validation. + */ + @Test + public void spannerToCassandraSourceAllDataTypeConversionTest() + throws InterruptedException, IOException, MultipleFailureException { + assertThatPipeline(jobInfo).isRunning(); + writeAllDataTypeRowsInSpanner(); + assertAllDataTypeRowsInCassandraDB(); + } + + /** + * Tests the conversion of string data types from Spanner to actual data type in Cassandra. + * + *

This test ensures that string-based data types are correctly written to Spanner and + * subsequently retrieved from Cassandra, verifying data integrity and conversion accuracy. + * + * @throws InterruptedException if the thread is interrupted during execution. + * @throws IOException if an I/O error occurs during the test execution. + * @throws MultipleFailureException if multiple assertions fail during validation. + */ + @Test + public void spannerToCassandraSourceDataTypeStringConversionTest() + throws InterruptedException, IOException, MultipleFailureException { + assertThatPipeline(jobInfo).isRunning(); + writeAllRowsAsStringInSpanner(); + assertStringToActualRowsInCassandraDB(); + } + + /** + * Retrieves the total row count of a specified table in Cassandra. + * + *

This method executes a `SELECT COUNT(*)` query on the given table and returns the number of + * rows present in it. + * + * @param tableName the name of the table whose row count is to be retrieved. + * @return the total number of rows in the specified table. + * @throws RuntimeException if the query does not return a result. + */ + private long getRowCount(String tableName) { + String query = String.format("SELECT COUNT(*) FROM %s", tableName); + ResultSet resultSet = cassandraResourceManager.executeStatement(query); + Row row = resultSet.one(); + if (row != null) { + return row.getLong(0); + } else { + throw new RuntimeException("Query did not return a result for table: " + tableName); + } + } + + /** + * Writes basic rows to multiple tables in Google Cloud Spanner. + * + *

This method performs the following operations: + * + *

    + *
  • Inserts or updates a row in the "users" table with an ID of 1. + *
  • Inserts or updates a row in the "users2" table with an ID of 2. + *
  • Executes a transactionally buffered insert/update operation in the "users" table with an + * ID of 3, using a transaction tag for tracking. + *
+ * + * The transaction uses a Spanner client with a specific transaction tag + * ("txBy=forwardMigration"). + */ + private void writeBasicRowInSpanner() { + Mutation m1 = + Mutation.newInsertOrUpdateBuilder("users") + .set("id") + .to(1) + .set("full_name") + .to("A") + .set("from") + .to("B") + .build(); + spannerResourceManager.write(m1); + + Mutation m2 = + Mutation.newInsertOrUpdateBuilder("users2") + .set("id") + .to(2) + .set("full_name") + .to("BB") + .build(); + spannerResourceManager.write(m2); + + // Write a single record to Spanner for the given logical shard + // Add the record with the transaction tag as txBy= + SpannerConfig spannerConfig = + SpannerConfig.create() + .withProjectId(PROJECT) + .withInstanceId(spannerResourceManager.getInstanceId()) + .withDatabaseId(spannerResourceManager.getDatabaseId()); + SpannerAccessor spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); + spannerAccessor + .getDatabaseClient() + .readWriteTransaction( + Options.tag("txBy=forwardMigration"), + Options.priority(spannerConfig.getRpcPriority().get())) + .run( + (TransactionRunner.TransactionCallable) + transaction -> { + Mutation m3 = + Mutation.newInsertOrUpdateBuilder("users") + .set("id") + .to(3) + .set("full_name") + .to("GG") + .set("from") + .to("BB") + .build(); + transaction.buffer(m3); + return null; + }); + } + + /** + * Asserts that a basic row exists in the Cassandra database. + * + *

This method performs the following steps: + * + *

    + *
  • Waits for the condition that ensures one row exists in the Cassandra table {@code + * USER_TABLE}. + *
  • Retrieves and logs rows from the Cassandra table. + *
  • Checks if exactly one row is present in the table. + *
  • Verifies that the row contains expected values for columns: {@code id}, {@code + * full_name}, and {@code from}. + *
+ * + * @throws InterruptedException if the thread is interrupted while waiting for the row count + * condition. + * @throws RuntimeException if reading from the Cassandra table fails. + */ + private void assertBasicRowInCassandraDB() throws InterruptedException { + PipelineOperator.Result result = + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount(USER_TABLE) == 1); + assertThatResult(result).meetsConditions(); + Iterable rows; + try { + LOG.info("Reading from Cassandra table: {}", USER_TABLE); + rows = cassandraResourceManager.readTable(USER_TABLE); + LOG.info("Cassandra Rows: {}", rows.toString()); + } catch (Exception e) { + throw new RuntimeException("Failed to read from Cassandra table: " + USER_TABLE, e); + } + + assertThat(rows).hasSize(1); + + Row row = rows.iterator().next(); + LOG.info("Cassandra Row to Assert: {}", row.toString()); + assertThat(row.getInt("id")).isEqualTo(1); + assertThat(row.getString("full_name")).isEqualTo("A"); + assertThat(row.getString("from")).isEqualTo("B"); + } + + /** + * Writes a row containing all supported data types into the Spanner database. + * + *

This method creates and inserts a row into the {@code ALL_DATA_TYPES_TABLE} with various + * data types, including text, numerical, date/time, boolean, byte arrays, lists, sets, and maps. + * The values are set explicitly to ensure compatibility with Spanner's schema. + * + *

The following data types are included: + * + *

    + *
  • {@code varchar_column} - String + *
  • {@code tinyint_column} - Integer + *
  • {@code text_column} - String + *
  • {@code date_column} - Date + *
  • {@code smallint_column} - Integer + *
  • {@code mediumint_column} - Integer + *
  • {@code int_column} - Integer + *
  • {@code bigint_column} - Long + *
  • {@code float_column} - Float + *
  • {@code double_column} - Double + *
  • {@code decimal_column} - BigDecimal + *
  • {@code datetime_column} - Timestamp + *
  • {@code timestamp_column} - Timestamp + *
  • {@code time_column} - String + *
  • {@code year_column} - String + *
  • {@code char_column} - String + *
  • {@code tinytext_column} - String + *
  • {@code mediumtext_column} - String + *
  • {@code longtext_column} - String + *
  • {@code enum_column} - String + *
  • {@code bool_column} - Boolean + *
  • {@code other_bool_column} - Boolean + *
  • {@code bytes_column} - ByteArray + *
  • {@code list_text_column} - JSON List of Strings + *
  • {@code list_int_column} - JSON List of Integers + *
  • {@code frozen_list_bigint_column} - JSON List of Big Integers + *
  • {@code set_text_column} - JSON Set of Strings + *
  • {@code set_date_column} - JSON Set of Dates + *
  • {@code frozen_set_bool_column} - JSON Set of Booleans + *
  • {@code map_text_to_int_column} - JSON Map of Strings to Integers + *
  • {@code map_date_to_text_column} - JSON Map of Dates to Strings + *
  • {@code frozen_map_int_to_bool_column} - JSON Map of Integers to Booleans + *
  • {@code map_text_to_list_column} - JSON Map of Strings to Lists + *
  • {@code map_text_to_set_column} - JSON Map of Strings to Sets + *
  • {@code set_of_maps_column} - JSON Set of Maps + *
  • {@code list_of_sets_column} - JSON List of Sets + *
  • {@code frozen_map_text_to_list_column} - JSON Map of Strings to Lists + *
  • {@code frozen_map_text_to_set_column} - JSON Map of Strings to Sets + *
  • {@code frozen_set_of_maps_column} - JSON Set of Maps + *
  • {@code frozen_list_of_sets_column} - JSON List of Sets + *
  • {@code varint_column} - String (Varint Representation) + *
  • {@code inet_column} - String (IP Address Representation) + *
+ * + * @throws RuntimeException if writing to Spanner fails. + */ + private void writeAllDataTypeRowsInSpanner() { + Mutation mutation = + Mutation.newInsertOrUpdateBuilder(ALL_DATA_TYPES_TABLE) + .set("varchar_column") + .to("SampleVarchar") + .set("tinyint_column") + .to(127) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(Value.date(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(32767) + .set("mediumint_column") + .to(8388607) + .set("int_column") + .to(2147483647) + .set("bigint_column") + .to(9223372036854775807L) + .set("float_column") + .to(3.14159) + .set("double_column") + .to(2.718281828459045) + .set("decimal_column") + .to(new BigDecimal("12345.6789")) + .set("datetime_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(Value.timestamp(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(Value.bool(Boolean.TRUE)) + .set("other_bool_column") + .to(Value.bool(Boolean.FALSE)) + .set("bytes_column") + .to(Value.bytes(ByteArray.copyFrom("Hello world"))) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to("123456789") + .set("inet_column") + .to("192.168.1.10") + .build(); + + spannerResourceManager.write(mutation); + } + + /** + * Executes multiple assertions and collects all assertion failures. + * + *

This method takes a variable number of {@link Runnable} assertions and executes them + * sequentially. If any assertions fail, their errors are collected, and a {@link + * MultipleFailureException} is thrown containing all assertion errors. + * + *

Usage Example: + * + *

+   * assertAll(
+   *     () -> assertEquals(expectedValue, actualValue),
+   *     () -> assertNotNull(someObject),
+   *     () -> assertTrue(condition)
+   * );
+   * 
+ * + * @param assertions One or more assertions provided as {@link Runnable} lambdas. + * @throws MultipleFailureException if one or more assertions fail. + */ + private void assertAll(Runnable... assertions) throws MultipleFailureException { + for (Runnable assertion : assertions) { + try { + assertion.run(); + } catch (AssertionError e) { + assertionErrors.add(e); + } + } + if (!assertionErrors.isEmpty()) { + throw new MultipleFailureException(assertionErrors); + } + } + + /** + * Validates that all data type rows inserted in Spanner have been correctly migrated and stored + * in Cassandra. + * + *

This method ensures that the data in the Cassandra table {@code ALL_DATA_TYPES_TABLE} + * matches the expected values after migration. It waits for the pipeline to process the data, + * reads the data from Cassandra, and asserts all column values. + * + *

Assertions: + * + *

    + *
  • Basic Data Types - Ensures correct values for varchar, bigint, bool, char, date, + * datetime, decimal, double, float. + *
  • Collections - Validates frozen lists, sets, and maps including nested structures. + *
  • Lists and Sets - Ensures list and set columns contain expected elements. + *
  • Maps - Validates various map column structures including text-to-int, date-to-text, and + * list/set mappings. + *
+ * + *

Example Usage: + * + *

+   * assertAllDataTypeRowsInCassandraDB();
+   * 
+ * + * @throws InterruptedException if the thread is interrupted while waiting for pipeline execution. + * @throws MultipleFailureException if multiple assertion failures occur. + */ + private void assertAllDataTypeRowsInCassandraDB() + throws InterruptedException, MultipleFailureException { + PipelineOperator.Result result = + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), + () -> getRowCount(ALL_DATA_TYPES_TABLE) == 1); + assertThatResult(result).meetsConditions(); + Iterable rows; + try { + rows = cassandraResourceManager.readTable(ALL_DATA_TYPES_TABLE); + } catch (Exception e) { + throw new RuntimeException("Failed to read from Cassandra table: " + ALL_DATA_TYPES_TABLE, e); + } + + assertThat(rows).hasSize(1); + + Row row = rows.iterator().next(); + + assertThat(rows).hasSize(1); + assertAll( + // Basic Data Types + () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), + () -> assertThat(row.getLong("bigint_column")).isEqualTo(9223372036854775807L), + () -> assertThat(row.getBoolean("bool_column")).isTrue(), + () -> assertThat(row.getString("char_column")).isEqualTo("CHAR_DATA"), + () -> + assertThat(row.getLocalDate("date_column")) + .isEqualTo(java.time.LocalDate.of(2025, 1, 27)), + () -> + assertThat(row.getInstant("datetime_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("12345.6789")), + () -> assertThat(row.getDouble("double_column")).isEqualTo(2.718281828459045), + () -> assertThat(row.getFloat("float_column")).isEqualTo(3.14159f), + + // Collections (frozen, list, set, map) + () -> + assertThat(row.getList("frozen_list_bigint_column", Long.class)) + .isEqualTo(Arrays.asList(123456789012345L, 987654321012345L)), + () -> + assertThat(row.getSet("frozen_set_bool_column", Boolean.class)) + .isEqualTo(new HashSet<>(Arrays.asList(false, true))), + () -> + assertThat(row.getMap("frozen_map_int_to_bool_column", Integer.class, Boolean.class)) + .isEqualTo(Map.of(1, true, 2, false)), + () -> + assertThat(row.getMap("frozen_map_text_to_list_column", String.class, List.class)) + .isEqualTo(Map.of("fruits", Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("frozen_map_text_to_set_column", String.class, Set.class)) + .isEqualTo(Map.of("vegetables", new HashSet<>(Arrays.asList("carrot", "spinach")))), + () -> + assertThat(row.getSet("frozen_set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + + // Lists and Sets + () -> + assertThat(row.getList("list_int_column", Integer.class)) + .isEqualTo(Arrays.asList(1, 2, 3, 4, 5)), + () -> + assertThat(row.getList("list_text_column", String.class)) + .isEqualTo(Arrays.asList("apple", "banana", "cherry")), + () -> + assertThat(row.getList("list_of_sets_column", Set.class)) + .isEqualTo( + Arrays.asList( + new HashSet<>(Arrays.asList("apple", "banana")), + new HashSet<>(Arrays.asList("carrot", "spinach")))), + + // Maps + () -> + assertThat( + row.getMap("map_date_to_text_column", java.time.LocalDate.class, String.class)) + .isEqualTo( + Map.of( + java.time.LocalDate.parse("2025-01-27"), "event1", + java.time.LocalDate.parse("2025-02-01"), "event2")), + () -> + assertThat(row.getMap("map_text_to_int_column", String.class, Integer.class)) + .isEqualTo(Map.of("key1", 10, "key2", 20)), + () -> + assertThat(row.getMap("map_text_to_list_column", String.class, List.class)) + .isEqualTo( + Map.of( + "color", + Arrays.asList("red", "green"), + "fruit", + Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("map_text_to_set_column", String.class, Set.class)) + .isEqualTo( + Map.of( + "fruit", + new HashSet<>(Arrays.asList("apple", "banana")), + "vegetables", + new HashSet<>(Arrays.asList("carrot", "spinach")))), + + // Sets + () -> + assertThat(row.getSet("set_date_column", java.time.LocalDate.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + java.time.LocalDate.parse("2025-01-27"), + java.time.LocalDate.parse("2025-02-01")))), + () -> + assertThat(row.getSet("set_text_column", String.class)) + .isEqualTo(new HashSet<>(Arrays.asList("apple", "orange", "banana"))), + () -> + assertThat(row.getSet("set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + + // Other Basic Types + () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 32767), + () -> assertThat(row.getInt("mediumint_column")).isEqualTo(8388607), + () -> assertThat(row.getInt("int_column")).isEqualTo(2147483647), + () -> assertThat(row.getString("enum_column")).isEqualTo("OptionA"), + () -> assertThat(row.getString("year_column")).isEqualTo("2025"), + () -> + assertThat(row.getString("longtext_column")) + .isEqualTo( + "Very long text data that exceeds the medium text column length for long text."), + () -> assertThat(row.getString("tinytext_column")).isEqualTo("Short text for tinytext."), + () -> + assertThat(row.getString("mediumtext_column")) + .isEqualTo("Longer text data for mediumtext column."), + () -> + assertThat(row.getString("text_column")) + .isEqualTo("This is some sample text data for the text column."), + () -> + assertThat(row.getLocalTime("time_column")) + .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), + () -> + assertThat(row.getInstant("timestamp_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigInteger("varint_column")) + .isEqualTo(java.math.BigInteger.valueOf(123456789L))); + } + + /** + * Inserts multiple rows into the Spanner table {@code ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE}, + * ensuring that all values are stored as strings, regardless of their original data type. + * + *

This method writes sample data to the Spanner table, converting all numerical, boolean, and + * date/time values to their string representations. This ensures compatibility for scenarios + * requiring string-based storage. + * + *

Columns and Data Mapping: + * + *

    + *
  • Basic Types: Strings, numbers (converted to strings), booleans. + *
  • Complex Types: JSON representations for lists, sets, and maps. + *
  • Temporal Types: Date, datetime, timestamp values stored as strings. + *
+ */ + private void writeAllRowsAsStringInSpanner() { + Mutation m; + m = + Mutation.newInsertOrUpdateBuilder(ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE) + .set("varchar_column") + .to("SampleVarchar") + .set("tinyint_column") + .to(String.valueOf(127)) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(String.valueOf(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(String.valueOf(32767)) + .set("mediumint_column") + .to(String.valueOf(8388607)) + .set("int_column") + .to(String.valueOf(2147483647)) + .set("bigint_column") + .to(String.valueOf(9223372036854775807L)) + .set("float_column") + .to(String.valueOf(3.14159f)) + .set("double_column") + .to(String.valueOf(2.718281828459045)) + .set("decimal_column") + .to(new BigDecimal("12345.6789").toPlainString()) + .set("datetime_column") + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(String.valueOf(Boolean.TRUE)) + .set("other_bool_column") + .to(String.valueOf(Boolean.FALSE)) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to("123456789") + .build(); + + spannerResourceManager.write(m); + + m = + Mutation.newInsertOrUpdateBuilder(ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE) + .set("varchar_column") + .to("SampleVarchar2") + .set("tinyint_column") + .to(String.valueOf(127)) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(String.valueOf(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(String.valueOf(32767)) + .set("mediumint_column") + .to(String.valueOf(8388607)) + .set("int_column") + .to(String.valueOf(2147483647)) + .set("bigint_column") + .to(String.valueOf(9223372036854775807L)) + .set("float_column") + .to(String.valueOf(3.14159f)) + .set("double_column") + .to(String.valueOf(2.718281828459045)) + .set("decimal_column") + .to(new BigDecimal("12345.6789").toPlainString()) + .set("datetime_column") + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(String.valueOf(Boolean.TRUE)) + .set("other_bool_column") + .to(String.valueOf(Boolean.FALSE)) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to("123456789") + .build(); + + spannerResourceManager.write(m); + } + + /** + * Validates that string-based data stored in Spanner is correctly converted to its actual data + * types when retrieved from Cassandra. + * + *

This method ensures that values stored as strings in Spanner are properly transformed into + * their expected data types in Cassandra. It performs the following: + * + *

    + *
  • Waits for the migration process to complete. + *
  • Reads and verifies that two rows are present in Cassandra. + *
  • Checks specific column values to confirm correct data type conversion. + *
+ * + *

Assertions Performed: + * + *

    + *
  • Verifies that {@code varchar_column} retains its expected string value. + *
  • Confirms that {@code tinyint_column} is correctly converted to a {@code byte}. + *
+ * + * @throws MultipleFailureException if multiple assertions fail during validation. + */ + private void assertStringToActualRowsInCassandraDB() throws MultipleFailureException { + PipelineOperator.Result result = + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(15)), + () -> getRowCount(ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE) == 2); + assertThatResult(result).meetsConditions(); + + Iterable rows; + try { + rows = cassandraResourceManager.readTable(ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE); + } catch (Exception e) { + throw new RuntimeException( + "Failed to read from Cassandra table: " + ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE, e); + } + + assertThat(rows).hasSize(2); + Row row = rows.iterator().next(); + assertAll( + () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar2"), + () -> assertThat(row.getByte("tinyint_column")).isEqualTo((byte) 127)); + } +} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java deleted file mode 100644 index eb93ef5020..0000000000 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCassandraIT.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Copyright (C) 2025 Google LLC - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package com.google.cloud.teleport.v2.templates; - -import static com.google.common.truth.Truth.assertThat; -import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; -import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; - -import com.datastax.oss.driver.api.core.cql.ResultSet; -import com.datastax.oss.driver.api.core.cql.Row; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Options; -import com.google.cloud.spanner.TransactionRunner; -import com.google.cloud.teleport.metadata.SkipDirectRunnerTest; -import com.google.cloud.teleport.metadata.TemplateIntegrationTest; -import com.google.pubsub.v1.SubscriptionName; -import java.io.IOException; -import java.time.Duration; -import java.util.HashSet; -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.pubsub.PubsubResourceManager; -import org.apache.beam.it.gcp.spanner.SpannerResourceManager; -import org.apache.beam.it.gcp.storage.GcsResourceManager; -import org.apache.beam.sdk.io.gcp.spanner.SpannerAccessor; -import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category({TemplateIntegrationTest.class, SkipDirectRunnerTest.class}) -@TemplateIntegrationTest(SpannerToSourceDb.class) -@RunWith(JUnit4.class) -public class SpannerToSourceDbCassandraIT extends SpannerToCassandraDbITBase { - - private static final Logger LOG = LoggerFactory.getLogger(SpannerToSourceDbCassandraIT.class); - - private static final String SPANNER_DDL_RESOURCE = - "SpannerToCassandraSourceIT/spanner-schema.sql"; - private static final String CASSANDRA_SCHEMA_FILE_RESOURCE = - "SpannerToCassandraSourceIT/cassandra-schema.sql"; - private static final String CASSANDRA_CONFIG_FILE_RESOURCE = - "SpannerToCassandraSourceIT/cassandra-config-template.conf"; - - private static final String TABLE = "Users"; - private static final HashSet testInstances = new HashSet<>(); - private static PipelineLauncher.LaunchInfo jobInfo; - public static SpannerResourceManager spannerResourceManager; - private static SpannerResourceManager spannerMetadataResourceManager; - public static CassandraSharedResourceManager cassandraResourceManager; - private static GcsResourceManager gcsResourceManager; - private static PubsubResourceManager pubsubResourceManager; - private SubscriptionName subscriptionName; - - /** - * Setup resource managers and Launch dataflow job once during the execution of this test class. - * - * @throws IOException - */ - @Before - public void setUp() throws IOException { - skipBaseCleanup = true; - synchronized (SpannerToSourceDbCassandraIT.class) { - testInstances.add(this); - if (jobInfo == null) { - spannerResourceManager = createSpannerDatabase(SPANNER_DDL_RESOURCE); - spannerMetadataResourceManager = createSpannerMetadataDatabase(); - - cassandraResourceManager = generateKeyspaceAndBuildCassandraResource(); - gcsResourceManager = - GcsResourceManager.builder(artifactBucketName, getClass().getSimpleName(), credentials) - .build(); - createAndUploadCassandraConfigToGcs( - gcsResourceManager, cassandraResourceManager, CASSANDRA_CONFIG_FILE_RESOURCE); - createCassandraSchema(cassandraResourceManager, CASSANDRA_SCHEMA_FILE_RESOURCE); - pubsubResourceManager = setUpPubSubResourceManager(); - subscriptionName = - createPubsubResources( - getClass().getSimpleName(), - pubsubResourceManager, - getGcsPath("dlq", gcsResourceManager).replace("gs://" + artifactBucketName, "")); - jobInfo = - launchDataflowJob( - gcsResourceManager, - spannerResourceManager, - spannerMetadataResourceManager, - subscriptionName.toString(), - null, - null, - null, - null, - null); - } - } - } - - /** - * Cleanup dataflow job and all the resources and resource managers. - * - * @throws IOException - */ - @AfterClass - public static void cleanUp() throws IOException { - for (SpannerToSourceDbCassandraIT instance : testInstances) { - instance.tearDownBase(); - } - ResourceManagerUtils.cleanResources( - spannerResourceManager, - cassandraResourceManager, - spannerMetadataResourceManager, - gcsResourceManager, - pubsubResourceManager); - } - - @Test - public void spannerToCasandraSourceDbBasic() throws InterruptedException, IOException { - assertThatPipeline(jobInfo).isRunning(); - writeRowInSpanner(); - assertRowInCassandraDB(); - } - - private long getRowCount() { - String query = String.format("SELECT COUNT(*) FROM %s", TABLE); - ResultSet resultSet = cassandraResourceManager.executeStatement(query); - Row row = resultSet.one(); - if (row != null) { - return row.getLong(0); - } else { - throw new RuntimeException("Query did not return a result for table: " + TABLE); - } - } - - private void writeRowInSpanner() { - Mutation m1 = - Mutation.newInsertOrUpdateBuilder("users") - .set("id") - .to(1) - .set("full_name") - .to("A") - .set("from") - .to("B") - .build(); - spannerResourceManager.write(m1); - - Mutation m2 = - Mutation.newInsertOrUpdateBuilder("users2") - .set("id") - .to(2) - .set("full_name") - .to("BB") - .build(); - spannerResourceManager.write(m2); - - // Write a single record to Spanner for the given logical shard - // Add the record with the transaction tag as txBy= - SpannerConfig spannerConfig = - SpannerConfig.create() - .withProjectId(PROJECT) - .withInstanceId(spannerResourceManager.getInstanceId()) - .withDatabaseId(spannerResourceManager.getDatabaseId()); - SpannerAccessor spannerAccessor = SpannerAccessor.getOrCreate(spannerConfig); - spannerAccessor - .getDatabaseClient() - .readWriteTransaction( - Options.tag("txBy=forwardMigration"), - Options.priority(spannerConfig.getRpcPriority().get())) - .run( - (TransactionRunner.TransactionCallable) - transaction -> { - Mutation m3 = - Mutation.newInsertOrUpdateBuilder("users") - .set("id") - .to(3) - .set("full_name") - .to("GG") - .set("from") - .to("BB") - .build(); - transaction.buffer(m3); - return null; - }); - } - - private void assertRowInCassandraDB() throws InterruptedException { - PipelineOperator.Result result = - pipelineOperator() - .waitForCondition( - createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount() == 1); - assertThatResult(result).meetsConditions(); - Iterable rows; - try { - LOG.info("Reading from Cassandra table: {}", TABLE); - rows = cassandraResourceManager.readTable(TABLE); - LOG.info("Cassandra Rows: {}", rows.toString()); - } catch (Exception e) { - throw new RuntimeException("Failed to read from Cassandra table: " + TABLE, e); - } - - assertThat(rows).hasSize(1); - - Row row = rows.iterator().next(); - LOG.info("Cassandra Row to Assert: {}", row.toString()); - assertThat(row.getInt("id")).isEqualTo(1); - assertThat(row.getString("full_name")).isEqualTo("A"); - assertThat(row.getString("from")).isEqualTo("B"); - } -} diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java index e58488283d..6a9b87291f 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java @@ -123,6 +123,7 @@ public void setUp() throws IOException, InterruptedException { "input/customShard.jar", "com.custom.CustomShardIdFetcherForIT", null, + null, null); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java index dab4dc27d8..b8b7bbec4e 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java @@ -132,7 +132,8 @@ public void setUp() throws IOException, InterruptedException { null, null, null, - customTransformation); + customTransformation, + null); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java index dbd023cdef..2a701cda02 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java @@ -118,6 +118,7 @@ public void setUp() throws IOException { null, null, null, + null, null); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java index 7c3ad39760..d1b960e816 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java @@ -114,6 +114,7 @@ public void setUp() throws IOException { null, null, null, + null, null); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java index 64d15895cd..ee582179de 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java @@ -25,14 +25,22 @@ import com.google.gson.JsonObject; import com.google.pubsub.v1.SubscriptionName; import com.google.pubsub.v1.TopicName; +import java.io.BufferedReader; +import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; +import java.time.format.DateTimeFormatter; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.utils.IORedirectUtil; import org.apache.beam.it.common.utils.PipelineUtils; +import org.apache.beam.it.common.utils.ResourceManagerUtils; import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.artifacts.utils.ArtifactUtils; import org.apache.beam.it.gcp.pubsub.PubsubResourceManager; @@ -53,14 +61,27 @@ protected SpannerResourceManager createSpannerDatabase(String spannerSchemaFile) SpannerResourceManager.builder("rr-main-" + testName, PROJECT, REGION) .maybeUseStaticInstance() .build(); - String ddl = - String.join( - " ", - Resources.readLines(Resources.getResource(spannerSchemaFile), StandardCharsets.UTF_8)); - ddl = ddl.trim(); - String[] ddls = ddl.split(";"); + + String ddl; + try (InputStream inputStream = + Thread.currentThread().getContextClassLoader().getResourceAsStream(spannerSchemaFile)) { + if (inputStream == null) { + throw new FileNotFoundException("Resource file not found: " + spannerSchemaFile); + } + try (BufferedReader reader = + new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { + ddl = reader.lines().collect(Collectors.joining("\n")); + } + } + + if (ddl.isBlank()) { + throw new IllegalStateException("DDL file is empty: " + spannerSchemaFile); + } + + String[] ddls = ddl.trim().split(";"); for (String d : ddls) { - if (!d.isBlank()) { + d = d.trim(); + if (!d.isEmpty()) { spannerResourceManager.executeDdlStatement(d); } } @@ -116,6 +137,68 @@ protected void createAndUploadShardConfigToGcs( gcsResourceManager.createArtifact("input/shard.json", shardFileContents); } + protected CassandraResourceManager generateKeyspaceAndBuildCassandraResource() { + String keyspaceName = + ResourceManagerUtils.generateResourceId( + testName, + Pattern.compile("[/\\\\. \"\u0000$]"), + "-", + 27, + DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss-SSSSSS")) + .replace('-', '_'); + if (keyspaceName.length() > 48) { + keyspaceName = keyspaceName.substring(0, 48); + } + return CassandraResourceManager.builder(testName) + .setKeyspaceName(keyspaceName) + .sePreGeneratedKeyspaceName(true) + .build(); + } + + protected void createCassandraSchema( + CassandraResourceManager cassandraResourceManager, String cassandraSchemaFile) + throws IOException { + String ddl = + String.join( + " ", + Resources.readLines( + Resources.getResource(cassandraSchemaFile), StandardCharsets.UTF_8)); + ddl = ddl.trim(); + String[] ddls = ddl.split(";"); + for (String d : ddls) { + if (!d.isBlank()) { + cassandraResourceManager.execute(d); + } + } + } + + public void createAndUploadCassandraConfigToGcs( + GcsResourceManager gcsResourceManager, + CassandraResourceManager cassandraResourceManagers, + String cassandraConfigFile) + throws IOException { + + String host = cassandraResourceManagers.getHost(); + int port = cassandraResourceManagers.getPort(); + String keyspaceName = cassandraResourceManagers.getKeyspaceName(); + String cassandraConfigContents; + try (InputStream inputStream = + Thread.currentThread().getContextClassLoader().getResourceAsStream(cassandraConfigFile)) { + if (inputStream == null) { + throw new FileNotFoundException("Resource file not found: " + cassandraConfigFile); + } + cassandraConfigContents = new String(inputStream.readAllBytes(), StandardCharsets.UTF_8); + } + + cassandraConfigContents = + cassandraConfigContents + .replace("##host##", host) + .replace("##port##", Integer.toString(port)) + .replace("##keyspace##", keyspaceName); + + gcsResourceManager.createArtifact("input/cassandra-config.conf", cassandraConfigContents); + } + public PipelineLauncher.LaunchInfo launchDataflowJob( GcsResourceManager gcsResourceManager, SpannerResourceManager spannerResourceManager, @@ -125,9 +208,9 @@ public PipelineLauncher.LaunchInfo launchDataflowJob( String shardingCustomJarPath, String shardingCustomClassName, String sourceDbTimezoneOffset, - CustomTransformation customTransformation) + CustomTransformation customTransformation, + String sourceType) throws IOException { - // default parameters Map params = new HashMap<>() { @@ -138,13 +221,18 @@ public PipelineLauncher.LaunchInfo launchDataflowJob( put("spannerProjectId", PROJECT); put("metadataDatabase", spannerMetadataResourceManager.getDatabaseId()); put("metadataInstance", spannerMetadataResourceManager.getInstanceId()); - put("sourceShardsFilePath", getGcsPath("input/shard.json", gcsResourceManager)); + put( + "sourceShardsFilePath", + getGcsPath( + sourceType != null ? "input/cassandra-config.conf" : "input/shard.json", + gcsResourceManager)); put("changeStreamName", "allstream"); put("dlqGcsPubSubSubscription", subscriptionName); put("deadLetterQueueDirectory", getGcsPath("dlq", gcsResourceManager)); put("maxShardConnections", "5"); put("maxNumWorkers", "1"); put("numWorkers", "1"); + put("sourceType", sourceType); } }; diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java index 1f5acdc952..cf71cd2c3e 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java @@ -124,6 +124,7 @@ public void setUp() throws IOException { null, null, null, + null, null); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java index 1ab2d78b49..43cee80730 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java @@ -113,6 +113,7 @@ public void setUp() throws IOException { null, null, "+10:00", + null, null); } } diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf deleted file mode 100644 index 97486a8de0..0000000000 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-config-template.conf +++ /dev/null @@ -1,12 +0,0 @@ - # Configuration for the DataStax Java driver for Apache Cassandra®. - # This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md. - # This file is meant to be used only in unit tests to test loading configuration from file. - # DO NOT USE FOR PRODUCTION. - - datastax-java-driver { - basic.contact-points = ["##host##:##port##"] - basic.session-keyspace = "##keyspace##" - basic.load-balancing-policy { - local-datacenter = "datacenter1" - } - } diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql deleted file mode 100644 index 1a9d6a37ca..0000000000 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/cassandra-schema.sql +++ /dev/null @@ -1,43 +0,0 @@ -CREATE TABLE AllDatatypeColumns ( - varchar_column text PRIMARY KEY, - tinyint_column tinyint, - text_column text, - date_column date, - smallint_column smallint, - mediumint_column int, - int_column int, - bigint_column bigint, - float_column float, - double_column double, - decimal_column decimal, - datetime_column timestamp, - timestamp_column timestamp, - time_column time, - year_column text, - char_column text, - tinytext_column text, - mediumtext_column text, - longtext_column text, - enum_column text, - bool_column boolean, - other_bool_column boolean, - bytes_column BLOB, - list_text_column list, - list_int_column list, - frozen_list_bigint_column frozen>, - set_text_column set, - set_date_column set, - frozen_set_bool_column frozen>, - map_text_to_int_column map, - map_date_to_text_column map, - frozen_map_int_to_bool_column frozen>, - map_text_to_list_column map>>, - map_text_to_set_column map>>, - set_of_maps_column set>>, - list_of_sets_column list>>, - frozen_map_text_to_list_column map>>, - frozen_map_text_to_set_column map>>, - frozen_set_of_maps_column set>>, - frozen_list_of_sets_column list>>, - varint_column varint -); diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql deleted file mode 100644 index 4cb32f2778..0000000000 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceDbDatatypeIT/spanner-schema.sql +++ /dev/null @@ -1,49 +0,0 @@ -CREATE TABLE IF NOT EXISTS alldatatypecolumns ( - varchar_column STRING(20) NOT NULL, - tinyint_column INT64, - text_column STRING(MAX), - date_column DATE, - smallint_column INT64, - mediumint_column INT64, - int_column INT64, - bigint_column INT64, - float_column FLOAT64, - double_column FLOAT64, - decimal_column NUMERIC, - datetime_column TIMESTAMP, - timestamp_column TIMESTAMP, - time_column STRING(MAX), - year_column STRING(MAX), - char_column STRING(10), - tinytext_column STRING(MAX), - mediumtext_column STRING(MAX), - longtext_column STRING(MAX), - enum_column STRING(MAX), - bool_column BOOL, - other_bool_column BOOL, - bytes_column BYTES(MAX), - list_text_column JSON, - list_int_column JSON, - frozen_list_bigint_column JSON, - set_text_column JSON, - set_date_column JSON, - frozen_set_bool_column JSON, - map_text_to_int_column JSON, - map_date_to_text_column JSON, - frozen_map_int_to_bool_column JSON, - map_text_to_list_column JSON, - map_text_to_set_column JSON, - set_of_maps_column JSON, - list_of_sets_column JSON, - frozen_map_text_to_list_column JSON, - frozen_map_text_to_set_column JSON, - frozen_set_of_maps_column JSON, - frozen_list_of_sets_column JSON, - varint_column BYTES(MAX) -) PRIMARY KEY(varchar_column); - -CREATE CHANGE STREAM allstream - FOR ALL OPTIONS ( - value_capture_type = 'NEW_ROW', - retention_period = '7d' - ); \ No newline at end of file diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql index e220267733..3fcc26b3bf 100644 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql @@ -2,4 +2,92 @@ CREATE TABLE users ( id int PRIMARY KEY, full_name text, "from" text +); + +CREATE TABLE AllDatatypeTransformation ( + varchar_column text PRIMARY KEY, + tinyint_column tinyint, + text_column text, + date_column date, + smallint_column smallint, + mediumint_column int, + int_column int, + bigint_column bigint, + float_column float, + double_column double, + decimal_column decimal, + datetime_column timestamp, + timestamp_column timestamp, + time_column time, + year_column text, + char_column text, + tinytext_column text, + mediumtext_column text, + longtext_column text, + enum_column text, + bool_column boolean, + other_bool_column boolean, + list_text_column list, + list_int_column list, + frozen_list_bigint_column frozen>, + set_text_column set, + set_date_column set, + frozen_set_bool_column frozen>, + map_text_to_int_column map, + map_date_to_text_column map, + frozen_map_int_to_bool_column frozen>, + map_text_to_list_column map>>, + map_text_to_set_column map>>, + set_of_maps_column set>>, + list_of_sets_column list>>, + frozen_map_text_to_list_column map>>, + frozen_map_text_to_set_column map>>, + frozen_set_of_maps_column set>>, + frozen_list_of_sets_column list>>, + varint_column varint +); + +CREATE TABLE AllDatatypeColumns ( + varchar_column text PRIMARY KEY, + tinyint_column tinyint, + text_column text, + date_column date, + smallint_column smallint, + mediumint_column int, + int_column int, + bigint_column bigint, + float_column float, + double_column double, + decimal_column decimal, + datetime_column timestamp, + timestamp_column timestamp, + time_column time, + year_column text, + char_column text, + tinytext_column text, + mediumtext_column text, + longtext_column text, + enum_column text, + bool_column boolean, + other_bool_column boolean, + bytes_column BLOB, + list_text_column list, + list_int_column list, + frozen_list_bigint_column frozen>, + set_text_column set, + set_date_column set, + frozen_set_bool_column frozen>, + map_text_to_int_column map, + map_date_to_text_column map, + frozen_map_int_to_bool_column frozen>, + map_text_to_list_column map>>, + map_text_to_set_column map>>, + set_of_maps_column set>>, + list_of_sets_column list>>, + frozen_map_text_to_list_column map>>, + frozen_map_text_to_set_column map>>, + frozen_set_of_maps_column set>>, + frozen_list_of_sets_column list>>, + varint_column varint, + inet_column INET ); \ No newline at end of file diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql index 5cedd597e1..0b6ae0b7a6 100644 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/spanner-schema.sql @@ -9,6 +9,94 @@ CREATE TABLE IF NOT EXISTS users2 ( full_name STRING(25), ) PRIMARY KEY(id); +CREATE TABLE IF NOT EXISTS alldatatypetransformation ( + varchar_column STRING(20) NOT NULL, + tinyint_column STRING(MAX), + text_column STRING(MAX), + date_column STRING(MAX), + smallint_column STRING(MAX), + mediumint_column STRING(MAX), + int_column STRING(MAX), + bigint_column STRING(MAX), + float_column STRING(MAX), + double_column STRING(MAX), + decimal_column STRING(MAX), + datetime_column STRING(MAX), + timestamp_column STRING(MAX), + time_column STRING(MAX), + year_column STRING(MAX), + char_column STRING(10), + tinytext_column STRING(MAX), + mediumtext_column STRING(MAX), + longtext_column STRING(MAX), + enum_column STRING(MAX), + bool_column STRING(MAX), + other_bool_column STRING(MAX), + list_text_column JSON, + list_int_column JSON, + frozen_list_bigint_column JSON, + set_text_column JSON, + set_date_column JSON, + frozen_set_bool_column JSON, + map_text_to_int_column JSON, + map_date_to_text_column JSON, + frozen_map_int_to_bool_column JSON, + map_text_to_list_column JSON, + map_text_to_set_column JSON, + set_of_maps_column JSON, + list_of_sets_column JSON, + frozen_map_text_to_list_column JSON, + frozen_map_text_to_set_column JSON, + frozen_set_of_maps_column JSON, + frozen_list_of_sets_column JSON, + varint_column STRING(MAX) +) PRIMARY KEY(varchar_column); + +CREATE TABLE IF NOT EXISTS alldatatypecolumns ( + varchar_column STRING(20) NOT NULL, + tinyint_column INT64, + text_column STRING(MAX), + date_column DATE, + smallint_column INT64, + mediumint_column INT64, + int_column INT64, + bigint_column INT64, + float_column FLOAT64, + double_column FLOAT64, + decimal_column NUMERIC, + datetime_column TIMESTAMP, + timestamp_column TIMESTAMP, + time_column STRING(MAX), + year_column STRING(MAX), + char_column STRING(10), + tinytext_column STRING(MAX), + mediumtext_column STRING(MAX), + longtext_column STRING(MAX), + enum_column STRING(MAX), + bool_column BOOL, + other_bool_column BOOL, + bytes_column BYTES(MAX), + list_text_column JSON, + list_int_column JSON, + frozen_list_bigint_column JSON, + set_text_column JSON, + set_date_column JSON, + frozen_set_bool_column JSON, + map_text_to_int_column JSON, + map_date_to_text_column JSON, + frozen_map_int_to_bool_column JSON, + map_text_to_list_column JSON, + map_text_to_set_column JSON, + set_of_maps_column JSON, + list_of_sets_column JSON, + frozen_map_text_to_list_column JSON, + frozen_map_text_to_set_column JSON, + frozen_set_of_maps_column JSON, + frozen_list_of_sets_column JSON, + varint_column STRING(MAX), + inet_column STRING(MAX) +) PRIMARY KEY(varchar_column); + CREATE CHANGE STREAM allstream FOR ALL OPTIONS ( value_capture_type = 'NEW_ROW', From 5d52dead1818aeb4fa0128c32a9654c1952c7099 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Wed, 5 Feb 2025 16:05:24 +0530 Subject: [PATCH 13/19] Handle IT for MYSQL --- .../teleport/v2/templates/SpannerToCassandraSourceDbIT.java | 3 ++- .../v2/templates/SpannerToSourceDbCustomShardIT.java | 3 ++- .../templates/SpannerToSourceDbCustomTransformationIT.java | 3 ++- .../teleport/v2/templates/SpannerToSourceDbDatatypeIT.java | 3 ++- .../cloud/teleport/v2/templates/SpannerToSourceDbIT.java | 3 ++- .../teleport/v2/templates/SpannerToSourceDbITBase.java | 6 +++++- .../templates/SpannerToSourceDbInterleaveMultiShardIT.java | 3 ++- .../teleport/v2/templates/SpannerToSourceDbTimezoneIT.java | 3 ++- 8 files changed, 19 insertions(+), 8 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java index f16c24c0c0..9afddda797 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.CASSANDRA_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -123,7 +124,7 @@ public void setUp() throws IOException { null, null, null, - "cassandra"); + CASSANDRA_SOURCE_TYPE); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java index 6a9b87291f..fc5f43057e 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomShardIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -124,7 +125,7 @@ public void setUp() throws IOException, InterruptedException { "com.custom.CustomShardIdFetcherForIT", null, null, - null); + MYSQL_SOURCE_TYPE); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java index b8b7bbec4e..3d7da5a2bd 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbCustomTransformationIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -133,7 +134,7 @@ public void setUp() throws IOException, InterruptedException { null, null, customTransformation, - null); + MYSQL_SOURCE_TYPE); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java index 2a701cda02..4ffcc23fbf 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbDatatypeIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -119,7 +120,7 @@ public void setUp() throws IOException { null, null, null, - null); + MYSQL_SOURCE_TYPE); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java index d1b960e816..286d863865 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -115,7 +116,7 @@ public void setUp() throws IOException { null, null, null, - null); + MYSQL_SOURCE_TYPE); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java index ee582179de..8260332eec 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; @@ -35,6 +36,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.beam.it.common.PipelineLauncher; @@ -224,7 +226,9 @@ public PipelineLauncher.LaunchInfo launchDataflowJob( put( "sourceShardsFilePath", getGcsPath( - sourceType != null ? "input/cassandra-config.conf" : "input/shard.json", + !Objects.equals(sourceType, MYSQL_SOURCE_TYPE) + ? "input/cassandra-config.conf" + : "input/shard.json", gcsResourceManager)); put("changeStreamName", "allstream"); put("dlqGcsPubSubSubscription", subscriptionName); diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java index cf71cd2c3e..915afbaca1 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbInterleaveMultiShardIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -125,7 +126,7 @@ public void setUp() throws IOException { null, null, null, - null); + MYSQL_SOURCE_TYPE); } } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java index 43cee80730..d533fe82ee 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbTimezoneIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -114,7 +115,7 @@ public void setUp() throws IOException { null, "+10:00", null, - null); + MYSQL_SOURCE_TYPE); } } } From 5eb9f20cc8992177d8d2ae71dc72af46137dd656 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Thu, 6 Feb 2025 10:37:50 +0530 Subject: [PATCH 14/19] Added ByteBuffer TO BigInteger --- .../v2/templates/dbutils/dml/CassandraTypeHandler.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java index ca4b5e6409..544adc6728 100644 --- a/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java +++ b/v2/spanner-to-sourcedb/src/main/java/com/google/cloud/teleport/v2/templates/dbutils/dml/CassandraTypeHandler.java @@ -122,10 +122,7 @@ private static BigInteger handleCassandraVarintType(Object value) { if (value instanceof byte[]) { return new BigInteger((byte[]) value); } else if (value instanceof ByteBuffer) { - ByteBuffer byteBuffer = (ByteBuffer) value; - byte[] byteArray = new byte[byteBuffer.remaining()]; - byteBuffer.get(byteArray); - return new BigInteger(byteArray); + return new BigInteger(((ByteBuffer) value).array()); } return new BigInteger(value.toString()); } From aae8d8712f5087ced100b76ffce397d2c04494a8 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Fri, 7 Feb 2025 18:07:02 +0530 Subject: [PATCH 15/19] Cassandra rr custom transfornation it test (#96) --- .../templates/CassandraResourceManager.java | 95 +------- .../SpannerToCassandraSourceDbIT.java | 206 ++++++++++++------ .../v2/templates/SpannerToSourceDbITBase.java | 25 +-- 3 files changed, 145 insertions(+), 181 deletions(-) diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java index ab1dbf39c8..46c8d5b264 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java @@ -25,8 +25,6 @@ import java.net.InetSocketAddress; import java.time.Duration; import java.time.format.DateTimeFormatter; -import java.util.List; -import java.util.Map; import java.util.regex.Pattern; import org.apache.beam.it.common.ResourceManager; import org.apache.beam.it.common.utils.ExceptionUtils; @@ -82,12 +80,9 @@ private CassandraResourceManager(Builder builder) { CassandraResourceManager( @Nullable CqlSession cassandraClient, CassandraContainer container, Builder builder) { super(container, builder); - // we are trying to handle userDefined KeyspaceName name without usingStatic Container - this.usingStaticDatabase = builder.keyspaceName != null && !builder.preGeneratedKeyspaceName; + this.usingStaticDatabase = builder.keyspaceName != null; this.keyspaceName = - usingStaticDatabase || builder.preGeneratedKeyspaceName - ? builder.keyspaceName - : generateKeyspaceName(builder.testId); + usingStaticDatabase ? builder.keyspaceName : generateKeyspaceName(builder.testId); this.cassandraClient = cassandraClient == null ? CqlSession.builder() @@ -176,54 +171,26 @@ public synchronized void execute(String statement) { } /** - * Inserts the given Documents into a collection. - * - *

Note: Implementations may do collection creation here, if one does not already exist. - * - * @param tableName The name of the collection to insert the documents into. - * @param documents A list of documents to insert into the collection. - * @return A boolean indicating whether the Documents were inserted successfully. - * @throws IllegalArgumentException if there is an error inserting the documents. - */ - public synchronized boolean insertDocuments(String tableName, List> documents) - throws IllegalArgumentException { - LOG.info( - "Attempting to write {} documents to {}.{}.", documents.size(), keyspaceName, tableName); - - try { - for (Map document : documents) { - executeStatement(createInsertStatement(tableName, document)); - } - } catch (Exception e) { - throw new IllegalArgumentException("Error inserting documents.", e); - } - - LOG.info("Successfully wrote {} documents to {}.{}", documents.size(), keyspaceName, tableName); - - return true; - } - - /** - * Reads all the Documents in a collection. + * Reads all the rows in a collection. * * @param tableName The name of the collection to read from. - * @return An iterable of all the Documents in the collection. + * @return An iterable of all the rows in the collection. * @throws IllegalArgumentException if there is an error reading the collection. */ public synchronized Iterable readTable(String tableName) throws IllegalArgumentException { LOG.info("Reading all documents from {}.{}", keyspaceName, tableName); - Iterable documents; + Iterable rows; try { ResultSet resultSet = executeStatement(String.format("SELECT * FROM %s", tableName)); - documents = resultSet.all(); + rows = resultSet.all(); } catch (Exception e) { throw new IllegalArgumentException("Error reading table.", e); } LOG.info("Successfully loaded documents from {}.{}", keyspaceName, tableName); - return documents; + return rows; } @Override @@ -238,8 +205,6 @@ public synchronized void cleanupAll() { executeStatement(String.format("DROP KEYSPACE IF EXISTS %s", this.keyspaceName)); } catch (Exception e) { LOG.error("Failed to drop Cassandra keyspace {}.", keyspaceName, e); - - // Only bubble exception if the cause is not timeout or does not exist if (!ExceptionUtils.containsType(e, DriverTimeoutException.class) && !ExceptionUtils.containsMessage(e, "does not exist")) { producedError = true; @@ -247,7 +212,6 @@ public synchronized void cleanupAll() { } } - // Next, try to close the Cassandra client connection try { cassandraClient.close(); } catch (Exception e) { @@ -255,7 +219,6 @@ public synchronized void cleanupAll() { producedError = true; } - // Throw Exception at the end if there were any errors if (producedError) { throw new IllegalArgumentException("Failed to delete resources. Check above for errors."); } @@ -265,31 +228,6 @@ public synchronized void cleanupAll() { LOG.info("Cassandra manager successfully cleaned up."); } - private String createInsertStatement(String tableName, Map map) { - StringBuilder columns = new StringBuilder(); - StringBuilder values = new StringBuilder(); - - for (Map.Entry entry : map.entrySet()) { - columns.append(entry.getKey()).append(", "); - - // add quotes around strings - if (entry.getValue() instanceof String) { - values.append("'").append(entry.getValue()).append("'"); - } else { - values.append(entry.getValue()); - } - values.append(", "); - } - - // Remove trailing comma and space - if (!map.isEmpty()) { - columns.delete(columns.length() - 2, columns.length()); - values.delete(values.length() - 2, values.length()); - } - - return String.format("INSERT INTO %s (%s) VALUES (%s)", tableName, columns, values); - } - private static RetryPolicy buildRetryPolicy() { return RetryPolicy.builder() .withMaxRetries(5) @@ -304,8 +242,6 @@ public static final class Builder private @Nullable String keyspaceName; - private @Nullable boolean preGeneratedKeyspaceName; - private Builder(String testId) { super(testId, DEFAULT_CASSANDRA_CONTAINER_NAME, DEFAULT_CASSANDRA_CONTAINER_TAG); this.keyspaceName = null; @@ -327,23 +263,6 @@ public Builder setKeyspaceName(String keyspaceName) { return this; } - /** - * Sets the preGeneratedKeyspaceName to that of a static database instance. Use this method only - * when attempting to operate on a pre-existing Cassandra database. - * - *

Note: if a database name is set, and a static Cassandra server is being used - * (useStaticContainer() is also called on the builder), then a database will be created on the - * static server if it does not exist, and it will not be removed when cleanupAll() is called on - * the CassandraResourceManager. - * - * @param preGeneratedKeyspaceName The database name. - * @return this builder object with the database is need to create. - */ - public Builder sePreGeneratedKeyspaceName(boolean preGeneratedKeyspaceName) { - this.preGeneratedKeyspaceName = preGeneratedKeyspaceName; - return this; - } - @Override public CassandraResourceManager build() { return new CassandraResourceManager(this); diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java index 9afddda797..35cf90fe22 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java @@ -15,7 +15,6 @@ */ package com.google.cloud.teleport.v2.templates; -import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.CASSANDRA_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -25,6 +24,7 @@ import com.google.cloud.ByteArray; import com.google.cloud.Date; import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Key; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Options; import com.google.cloud.spanner.TransactionRunner; @@ -34,6 +34,7 @@ import com.google.pubsub.v1.SubscriptionName; import java.io.IOException; import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -124,7 +125,7 @@ public void setUp() throws IOException { null, null, null, - CASSANDRA_SOURCE_TYPE); + "cassandra"); } } } @@ -163,6 +164,43 @@ public void spannerToCasandraSourceDbBasic() throws InterruptedException, IOExce assertBasicRowInCassandraDB(); } + /** + * Tests the data flow from Spanner to Cassandra. + * + *

This test ensures that a basic row is successfully deleted from Spanner and subsequently + * deleted in Cassandra, validating end-to-end data consistency. + * + * @throws InterruptedException if the thread is interrupted during execution. + * @throws IOException if an I/O error occurs during the test execution. + */ + @Test + public void spannerToCasandraSourceDbDeleteOperation() throws InterruptedException, IOException { + assertThatPipeline(jobInfo).isRunning(); + writeDeleteInSpanner(); + assertDeleteRowInCassandraDB(); + } + + /** De basic rows to multiple tables in Google Cloud Spanner. */ + private void writeDeleteInSpanner() { + Mutation m = Mutation.delete(USER_TABLE, Key.of(1, 3)); + spannerResourceManager.write(m); + } + + /** + * Asserts that delete the Cassandra database. + * + * @throws InterruptedException if the thread is interrupted while waiting for the row count + * condition. + * @throws RuntimeException if reading from the Cassandra table fails. + */ + private void assertDeleteRowInCassandraDB() throws InterruptedException { + PipelineOperator.Result result = + pipelineOperator() + .waitForCondition( + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount(USER_TABLE) == 0); + assertThatResult(result).meetsConditions(); + } + /** * Tests the data type conversion from Spanner to Cassandra. * @@ -335,53 +373,6 @@ private void assertBasicRowInCassandraDB() throws InterruptedException { * data types, including text, numerical, date/time, boolean, byte arrays, lists, sets, and maps. * The values are set explicitly to ensure compatibility with Spanner's schema. * - *

The following data types are included: - * - *

    - *
  • {@code varchar_column} - String - *
  • {@code tinyint_column} - Integer - *
  • {@code text_column} - String - *
  • {@code date_column} - Date - *
  • {@code smallint_column} - Integer - *
  • {@code mediumint_column} - Integer - *
  • {@code int_column} - Integer - *
  • {@code bigint_column} - Long - *
  • {@code float_column} - Float - *
  • {@code double_column} - Double - *
  • {@code decimal_column} - BigDecimal - *
  • {@code datetime_column} - Timestamp - *
  • {@code timestamp_column} - Timestamp - *
  • {@code time_column} - String - *
  • {@code year_column} - String - *
  • {@code char_column} - String - *
  • {@code tinytext_column} - String - *
  • {@code mediumtext_column} - String - *
  • {@code longtext_column} - String - *
  • {@code enum_column} - String - *
  • {@code bool_column} - Boolean - *
  • {@code other_bool_column} - Boolean - *
  • {@code bytes_column} - ByteArray - *
  • {@code list_text_column} - JSON List of Strings - *
  • {@code list_int_column} - JSON List of Integers - *
  • {@code frozen_list_bigint_column} - JSON List of Big Integers - *
  • {@code set_text_column} - JSON Set of Strings - *
  • {@code set_date_column} - JSON Set of Dates - *
  • {@code frozen_set_bool_column} - JSON Set of Booleans - *
  • {@code map_text_to_int_column} - JSON Map of Strings to Integers - *
  • {@code map_date_to_text_column} - JSON Map of Dates to Strings - *
  • {@code frozen_map_int_to_bool_column} - JSON Map of Integers to Booleans - *
  • {@code map_text_to_list_column} - JSON Map of Strings to Lists - *
  • {@code map_text_to_set_column} - JSON Map of Strings to Sets - *
  • {@code set_of_maps_column} - JSON Set of Maps - *
  • {@code list_of_sets_column} - JSON List of Sets - *
  • {@code frozen_map_text_to_list_column} - JSON Map of Strings to Lists - *
  • {@code frozen_map_text_to_set_column} - JSON Map of Strings to Sets - *
  • {@code frozen_set_of_maps_column} - JSON Set of Maps - *
  • {@code frozen_list_of_sets_column} - JSON List of Sets - *
  • {@code varint_column} - String (Varint Representation) - *
  • {@code inet_column} - String (IP Address Representation) - *
- * * @throws RuntimeException if writing to Spanner fails. */ private void writeAllDataTypeRowsInSpanner() { @@ -485,16 +476,6 @@ private void writeAllDataTypeRowsInSpanner() { * sequentially. If any assertions fail, their errors are collected, and a {@link * MultipleFailureException} is thrown containing all assertion errors. * - *

Usage Example: - * - *

-   * assertAll(
-   *     () -> assertEquals(expectedValue, actualValue),
-   *     () -> assertNotNull(someObject),
-   *     () -> assertTrue(condition)
-   * );
-   * 
- * * @param assertions One or more assertions provided as {@link Runnable} lambdas. * @throws MultipleFailureException if one or more assertions fail. */ @@ -530,12 +511,6 @@ private void assertAll(Runnable... assertions) throws MultipleFailureException { * list/set mappings. * * - *

Example Usage: - * - *

-   * assertAllDataTypeRowsInCassandraDB();
-   * 
- * * @throws InterruptedException if the thread is interrupted while waiting for pipeline execution. * @throws MultipleFailureException if multiple assertion failures occur. */ @@ -681,10 +656,13 @@ private void assertAllDataTypeRowsInCassandraDB() .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), () -> assertThat(row.getInstant("timestamp_column")) - .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.123456Z")), () -> assertThat(row.getBigInteger("varint_column")) - .isEqualTo(java.math.BigInteger.valueOf(123456789L))); + .isEqualTo(java.math.BigInteger.valueOf(123456789L)), + () -> + assertThat(row.getBytesUnsafe("bytes_column")) + .isEqualTo(ByteBuffer.wrap(ByteArray.copyFrom("Hello world").toByteArray()))); } /** @@ -732,7 +710,7 @@ private void writeAllRowsAsStringInSpanner() { .set("datetime_column") .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) .set("timestamp_column") - .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00.123456Z"))) .set("time_column") .to("12:30:00") .set("year_column") @@ -820,7 +798,95 @@ private void writeAllRowsAsStringInSpanner() { .set("datetime_column") .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) .set("timestamp_column") + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00.123456Z"))) + .set("time_column") + .to("12:30:00") + .set("year_column") + .to("2025") + .set("char_column") + .to("CHAR_DATA") + .set("tinytext_column") + .to("Short text for tinytext.") + .set("mediumtext_column") + .to("Longer text data for mediumtext column.") + .set("longtext_column") + .to("Very long text data that exceeds the medium text column length for long text.") + .set("enum_column") + .to("OptionA") + .set("bool_column") + .to(String.valueOf(Boolean.TRUE)) + .set("other_bool_column") + .to(String.valueOf(Boolean.FALSE)) + .set("list_text_column") + .to(Value.json("[\"apple\", \"banana\", \"cherry\"]")) + .set("list_int_column") + .to(Value.json("[1, 2, 3, 4, 5]")) + .set("frozen_list_bigint_column") + .to(Value.json("[123456789012345, 987654321012345]")) + .set("set_text_column") + .to(Value.json("[\"apple\", \"orange\", \"banana\"]")) + .set("set_date_column") + .to(Value.json("[\"2025-01-27\", \"2025-02-01\"]")) + .set("frozen_set_bool_column") + .to(Value.json("[true, false]")) + .set("map_text_to_int_column") + .to(Value.json("{\"key1\": 10, \"key2\": 20}")) + .set("map_date_to_text_column") + .to(Value.json("{\"2025-01-27\": \"event1\", \"2025-02-01\": \"event2\"}")) + .set("frozen_map_int_to_bool_column") + .to(Value.json("{\"1\": true, \"2\": false}")) + .set("map_text_to_list_column") + .to(Value.json("{\"fruit\": [\"apple\", \"banana\"], \"color\": [\"red\", \"green\"]}")) + .set("map_text_to_set_column") + .to( + Value.json( + "{\"fruit\": [\"apple\", \"banana\"], \"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("frozen_map_text_to_list_column") + .to(Value.json("{\"fruits\": [\"apple\", \"banana\"]}")) + .set("frozen_map_text_to_set_column") + .to(Value.json("{\"vegetables\": [\"carrot\", \"spinach\"]}")) + .set("frozen_set_of_maps_column") + .to(Value.json("[{\"key1\": 10, \"key2\": 20}, {\"keyA\": 5, \"keyB\": 10}]")) + .set("frozen_list_of_sets_column") + .to(Value.json("[[\"apple\", \"banana\"], [\"carrot\", \"spinach\"]]")) + .set("varint_column") + .to("123456789") + .build(); + + spannerResourceManager.write(m); + + m = + Mutation.newUpdateBuilder(ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE) + .set("varchar_column") + .to("SampleVarchar") + .set("tinyint_column") + .to(String.valueOf(122)) + .set("text_column") + .to("This is some sample text data for the text column.") + .set("date_column") + .to(String.valueOf(Date.fromJavaUtilDate(java.sql.Date.valueOf("2025-01-27")))) + .set("smallint_column") + .to(String.valueOf(32767)) + .set("mediumint_column") + .to(String.valueOf(8388607)) + .set("int_column") + .to(String.valueOf(2147483647)) + .set("bigint_column") + .to(String.valueOf(9223372036854775807L)) + .set("float_column") + .to(String.valueOf(3.14159f)) + .set("double_column") + .to(String.valueOf(2.718281828459045)) + .set("decimal_column") + .to(new BigDecimal("12345.6789").toPlainString()) + .set("datetime_column") .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00Z"))) + .set("timestamp_column") + .to(String.valueOf(Timestamp.parseTimestamp("2025-01-27T10:30:00.123456Z"))) .set("time_column") .to("12:30:00") .set("year_column") @@ -923,7 +989,7 @@ private void assertStringToActualRowsInCassandraDB() throws MultipleFailureExcep assertThat(rows).hasSize(2); Row row = rows.iterator().next(); assertAll( - () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar2"), - () -> assertThat(row.getByte("tinyint_column")).isEqualTo((byte) 127)); + () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), + () -> assertThat(row.getByte("tinyint_column")).isEqualTo((byte) 122)); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java index 8260332eec..9323254360 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java @@ -15,7 +15,6 @@ */ package com.google.cloud.teleport.v2.templates; -import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; @@ -32,17 +31,13 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; -import java.time.format.DateTimeFormatter; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.Objects; -import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.utils.IORedirectUtil; import org.apache.beam.it.common.utils.PipelineUtils; -import org.apache.beam.it.common.utils.ResourceManagerUtils; import org.apache.beam.it.gcp.TemplateTestBase; import org.apache.beam.it.gcp.artifacts.utils.ArtifactUtils; import org.apache.beam.it.gcp.pubsub.PubsubResourceManager; @@ -140,21 +135,7 @@ protected void createAndUploadShardConfigToGcs( } protected CassandraResourceManager generateKeyspaceAndBuildCassandraResource() { - String keyspaceName = - ResourceManagerUtils.generateResourceId( - testName, - Pattern.compile("[/\\\\. \"\u0000$]"), - "-", - 27, - DateTimeFormatter.ofPattern("yyyyMMdd-HHmmss-SSSSSS")) - .replace('-', '_'); - if (keyspaceName.length() > 48) { - keyspaceName = keyspaceName.substring(0, 48); - } - return CassandraResourceManager.builder(testName) - .setKeyspaceName(keyspaceName) - .sePreGeneratedKeyspaceName(true) - .build(); + return CassandraResourceManager.builder(testName).build(); } protected void createCassandraSchema( @@ -226,9 +207,7 @@ public PipelineLauncher.LaunchInfo launchDataflowJob( put( "sourceShardsFilePath", getGcsPath( - !Objects.equals(sourceType, MYSQL_SOURCE_TYPE) - ? "input/cassandra-config.conf" - : "input/shard.json", + sourceType != null ? "input/cassandra-config.conf" : "input/shard.json", gcsResourceManager)); put("changeStreamName", "allstream"); put("dlqGcsPubSubSubscription", subscriptionName); From 37040fb7c1a75fa2070c832a79ba4a786976b371 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Mon, 10 Feb 2025 14:15:15 +0530 Subject: [PATCH 16/19] PR Review Comments (#97) --- .../teleport/v2/utils/DatastreamToDML.java | 1 - .../v2/utils/DatastreamToPostgresDML.java | 55 ++++++ .../v2/utils/DatastreamToDMLTest.java | 82 +++++++- .../templates/CassandraResourceManager.java | 4 +- .../SpannerToCassandraSourceDbIT.java | 181 ++++++++++++++++-- .../v2/templates/SpannerToSourceDbITBase.java | 6 +- .../cassandra-schema.sql | 5 + 7 files changed, 306 insertions(+), 28 deletions(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java index 6a208e3579..a626c9237d 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToDML.java @@ -295,7 +295,6 @@ public String getValueSql(JsonNode rowObj, String columnName, Map elements = new ArrayList<>(); + if (arrayNode.isArray()) { + for (JsonNode element : arrayNode) { + if (element.has("elementValue")) { + JsonNode elementValue = element.get("elementValue"); + if (!elementValue.isNull()) { + elements.add(formatArrayElement(elementValue)); + } else { + elements.add(getNullValueSql()); + } + } else if (!element.isNull()) { + elements.add(formatArrayElement(element)); + } + } + } + return "ARRAY[" + String.join(",", elements) + "]"; + } catch (JsonProcessingException e) { + LOG.error("Error parsing JSON array: {}", jsonValue); + return getNullValueSql(); + } + } + + private String formatArrayElement(JsonNode element) { + if (element.isTextual()) { + return "\'" + cleanSql(element.textValue()) + "\'"; + } + return element.toString(); + } } diff --git a/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java b/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java index 585efd2fbe..79fb365155 100644 --- a/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java +++ b/v2/datastream-to-sql/src/test/java/com/google/cloud/teleport/v2/utils/DatastreamToDMLTest.java @@ -33,7 +33,7 @@ public class DatastreamToDMLTest { private static final Logger LOG = LoggerFactory.getLogger(DatastreamToDMLTest.class); - private String jsonString = + private static final String JSON_STRING = "{" + "\"text_column\":\"value\"," + "\"quoted_text_column\":\"Test Values: '!@#$%^\"," @@ -42,7 +42,7 @@ public class DatastreamToDMLTest { + "\"_metadata_table\":\"MY_TABLE$NAME\"" + "}"; - private JsonNode getRowObj() { + private JsonNode getRowObj(String jsonString) { ObjectMapper mapper = new ObjectMapper(); JsonNode rowObj; try { @@ -59,7 +59,7 @@ private JsonNode getRowObj() { */ @Test public void testGetValueSql() { - JsonNode rowObj = this.getRowObj(); + JsonNode rowObj = this.getRowObj(JSON_STRING); String expectedTextContent = "'value'"; String testSqlContent = @@ -82,6 +82,78 @@ public void testGetValueSql() { assertEquals(expectedNullByteTextContent, testNullByteSqlContent); } + /** + * Test whether {@link DatastreamToPostgresDML#getValueSql(JsonNode, String, Map)} converts array + * data into correct integer array syntax. + */ + @Test + public void testIntArrayWithNullTypeCoercion() { + String arrayJson = + "{\"number_array\": {" + + "\"nestedArray\": [" + + " {\"nestedArray\": null, \"elementValue\": null}," + + " {\"nestedArray\": null, \"elementValue\": 456}" + + "], \"elementValue\": null}}"; + JsonNode rowObj = this.getRowObj(arrayJson); + Map tableSchema = new HashMap<>(); + tableSchema.put("number_array", "_int4"); + DatastreamToPostgresDML dml = DatastreamToPostgresDML.of(null); + String expectedInt = "ARRAY[NULL,456]"; + + String actualInt = + DatastreamToPostgresDML.of(null).getValueSql(rowObj, "number_array", tableSchema); + + assertEquals(expectedInt, actualInt); + } + + /** + * Test whether {@link DatastreamToPostgresDML#getValueSql(JsonNode, String, Map)} converts array + * data into correct integer array syntax. + */ + @Test + public void testIntArrayTypeCoercion() { + String arrayJson = + "{\"number_array\": {" + + "\"nestedArray\": [" + + " {\"nestedArray\": null, \"elementValue\": 123}," + + " {\"nestedArray\": null, \"elementValue\": 456}" + + "], \"elementValue\": null}}"; + JsonNode rowObj = this.getRowObj(arrayJson); + Map tableSchema = new HashMap<>(); + tableSchema.put("number_array", "_int4"); + DatastreamToPostgresDML dml = DatastreamToPostgresDML.of(null); + String expectedInt = "ARRAY[123,456]"; + + String actualInt = + DatastreamToPostgresDML.of(null).getValueSql(rowObj, "number_array", tableSchema); + + assertEquals(expectedInt, actualInt); + } + + /** + * Test whether {@link DatastreamToPostgresDML#getValueSql(JsonNode, String, Map)} converts array + * data into correct text array syntax. + */ + @Test + public void testTextArrayTypeCoercion() { + String arrayJson = + "{\"text_array\": {" + + "\"nestedArray\": [" + + " {\"nestedArray\": null, \"elementValue\": \"apple\"}," + + " {\"nestedArray\": null, \"elementValue\": \"cherry\"}" + + "], \"elementValue\": null}}"; + JsonNode rowObj = this.getRowObj(arrayJson); + Map tableSchema = new HashMap<>(); + tableSchema.put("text_array", "_text"); + DatastreamToPostgresDML dml = DatastreamToPostgresDML.of(null); + String expectedInt = "ARRAY['apple','cherry']"; + + String actualInt = + DatastreamToPostgresDML.of(null).getValueSql(rowObj, "text_array", tableSchema); + + assertEquals(expectedInt, actualInt); + } + /** * Test whether {@link DatastreamToDML#getTargetSchemaName} converts the Oracle schema into the * correct Postgres schema. @@ -89,7 +161,7 @@ public void testGetValueSql() { @Test public void testGetPostgresSchemaName() { DatastreamToDML datastreamToDML = DatastreamToPostgresDML.of(null); - JsonNode rowObj = this.getRowObj(); + JsonNode rowObj = this.getRowObj(JSON_STRING); DatastreamRow row = DatastreamRow.of(rowObj); String expectedSchemaName = "my_schema"; @@ -104,7 +176,7 @@ public void testGetPostgresSchemaName() { @Test public void testGetPostgresTableName() { DatastreamToDML datastreamToDML = DatastreamToPostgresDML.of(null); - JsonNode rowObj = this.getRowObj(); + JsonNode rowObj = this.getRowObj(JSON_STRING); DatastreamRow row = DatastreamRow.of(rowObj); String expectedTableName = "my_table$name"; diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java index 46c8d5b264..e584175068 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/CassandraResourceManager.java @@ -178,7 +178,7 @@ public synchronized void execute(String statement) { * @throws IllegalArgumentException if there is an error reading the collection. */ public synchronized Iterable readTable(String tableName) throws IllegalArgumentException { - LOG.info("Reading all documents from {}.{}", keyspaceName, tableName); + LOG.info("Reading all rows from {}.{}", keyspaceName, tableName); Iterable rows; try { @@ -188,7 +188,7 @@ public synchronized Iterable readTable(String tableName) throws IllegalArgu throw new IllegalArgumentException("Error reading table.", e); } - LOG.info("Successfully loaded documents from {}.{}", keyspaceName, tableName); + LOG.info("Successfully loaded rows from {}.{}", keyspaceName, tableName); return rows; } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java index 35cf90fe22..496c10a560 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToCassandraSourceDbIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.CASSANDRA_SOURCE_TYPE; import static com.google.common.truth.Truth.assertThat; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatResult; @@ -24,7 +25,7 @@ import com.google.cloud.ByteArray; import com.google.cloud.Date; import com.google.cloud.Timestamp; -import com.google.cloud.spanner.Key; +import com.google.cloud.spanner.KeySet; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.Options; import com.google.cloud.spanner.TransactionRunner; @@ -75,6 +76,7 @@ public class SpannerToCassandraSourceDbIT extends SpannerToSourceDbITBase { "SpannerToCassandraSourceIT/cassandra-config-template.conf"; private static final String USER_TABLE = "Users"; + private static final String USER_TABLE_2 = "Users2"; private static final String ALL_DATA_TYPES_TABLE = "AllDatatypeColumns"; private static final String ALL_DATA_TYPES_CUSTOM_CONVERSION_TABLE = "AllDatatypeTransformation"; private static final HashSet testInstances = new HashSet<>(); @@ -125,7 +127,7 @@ public void setUp() throws IOException { null, null, null, - "cassandra"); + CASSANDRA_SOURCE_TYPE); } } } @@ -182,8 +184,19 @@ public void spannerToCasandraSourceDbDeleteOperation() throws InterruptedExcepti /** De basic rows to multiple tables in Google Cloud Spanner. */ private void writeDeleteInSpanner() { - Mutation m = Mutation.delete(USER_TABLE, Key.of(1, 3)); - spannerResourceManager.write(m); + + Mutation insertOrUpdateMutation = + Mutation.newInsertOrUpdateBuilder(USER_TABLE_2) + .set("id") + .to(4) + .set("full_name") + .to("GG") + .build(); + spannerResourceManager.write(insertOrUpdateMutation); + + KeySet allRows = KeySet.all(); + Mutation deleteAllMutation = Mutation.delete(USER_TABLE_2, allRows); + spannerResourceManager.write(deleteAllMutation); } /** @@ -275,7 +288,7 @@ private long getRowCount(String tableName) { */ private void writeBasicRowInSpanner() { Mutation m1 = - Mutation.newInsertOrUpdateBuilder("users") + Mutation.newInsertOrUpdateBuilder(USER_TABLE) .set("id") .to(1) .set("full_name") @@ -286,7 +299,7 @@ private void writeBasicRowInSpanner() { spannerResourceManager.write(m1); Mutation m2 = - Mutation.newInsertOrUpdateBuilder("users2") + Mutation.newInsertOrUpdateBuilder(USER_TABLE) .set("id") .to(2) .set("full_name") @@ -311,13 +324,11 @@ private void writeBasicRowInSpanner() { (TransactionRunner.TransactionCallable) transaction -> { Mutation m3 = - Mutation.newInsertOrUpdateBuilder("users") + Mutation.newInsertOrUpdateBuilder(USER_TABLE_2) .set("id") .to(3) .set("full_name") .to("GG") - .set("from") - .to("BB") .build(); transaction.buffer(m3); return null; @@ -346,8 +357,9 @@ private void assertBasicRowInCassandraDB() throws InterruptedException { PipelineOperator.Result result = pipelineOperator() .waitForCondition( - createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount(USER_TABLE) == 1); + createConfig(jobInfo, Duration.ofMinutes(10)), () -> getRowCount(USER_TABLE) == 2); assertThatResult(result).meetsConditions(); + Iterable rows; try { LOG.info("Reading from Cassandra table: {}", USER_TABLE); @@ -357,13 +369,20 @@ private void assertBasicRowInCassandraDB() throws InterruptedException { throw new RuntimeException("Failed to read from Cassandra table: " + USER_TABLE, e); } - assertThat(rows).hasSize(1); + assertThat(rows).hasSize(2); - Row row = rows.iterator().next(); - LOG.info("Cassandra Row to Assert: {}", row.toString()); - assertThat(row.getInt("id")).isEqualTo(1); - assertThat(row.getString("full_name")).isEqualTo("A"); - assertThat(row.getString("from")).isEqualTo("B"); + for (Row row : rows) { + LOG.info("Cassandra Row to Assert: {}", row.getFormattedContents()); + int id = row.getInt("id"); + if (id == 1) { + assertThat(row.getString("full_name")).isEqualTo("A"); + assertThat(row.getString("from")).isEqualTo("B"); + } else if (id == 2) { + assertThat(row.getString("full_name")).isEqualTo("BB"); + } else { + throw new AssertionError("Unexpected row ID found: " + id); + } + } } /** @@ -656,7 +675,7 @@ private void assertAllDataTypeRowsInCassandraDB() .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), () -> assertThat(row.getInstant("timestamp_column")) - .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.123456Z")), + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00Z")), () -> assertThat(row.getBigInteger("varint_column")) .isEqualTo(java.math.BigInteger.valueOf(123456789L)), @@ -988,8 +1007,132 @@ private void assertStringToActualRowsInCassandraDB() throws MultipleFailureExcep assertThat(rows).hasSize(2); Row row = rows.iterator().next(); + LOG.info("[AssertStringToActualRowsInCassandraDB] row: {}", row.getFormattedContents()); assertAll( - () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar"), - () -> assertThat(row.getByte("tinyint_column")).isEqualTo((byte) 122)); + () -> assertThat(row.getString("varchar_column")).isEqualTo("SampleVarchar2"), + () -> assertThat(row.getByte("tinyint_column")).isEqualTo((byte) 127), + () -> assertThat(row.getLong("bigint_column")).isEqualTo(9223372036854775807L), + () -> assertThat(row.getBoolean("bool_column")).isTrue(), + () -> assertThat(row.getString("char_column")).isEqualTo("CHAR_DATA"), + () -> + assertThat(row.getLocalDate("date_column")) + .isEqualTo(java.time.LocalDate.of(2025, 1, 27)), + () -> + assertThat(row.getInstant("datetime_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.000Z")), + () -> + assertThat(row.getBigDecimal("decimal_column")).isEqualTo(new BigDecimal("12345.6789")), + () -> assertThat(row.getDouble("double_column")).isEqualTo(2.718281828459045), + () -> assertThat(row.getFloat("float_column")).isEqualTo(3.14159f), + + // Collections (frozen, list, set, map) + () -> + assertThat(row.getList("frozen_list_bigint_column", Long.class)) + .isEqualTo(Arrays.asList(123456789012345L, 987654321012345L)), + () -> + assertThat(row.getSet("frozen_set_bool_column", Boolean.class)) + .isEqualTo(new HashSet<>(Arrays.asList(false, true))), + () -> + assertThat(row.getMap("frozen_map_int_to_bool_column", Integer.class, Boolean.class)) + .isEqualTo(Map.of(1, true, 2, false)), + () -> + assertThat(row.getMap("frozen_map_text_to_list_column", String.class, List.class)) + .isEqualTo(Map.of("fruits", Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("frozen_map_text_to_set_column", String.class, Set.class)) + .isEqualTo(Map.of("vegetables", new HashSet<>(Arrays.asList("carrot", "spinach")))), + () -> + assertThat(row.getSet("frozen_set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + + // Lists and Sets + () -> + assertThat(row.getList("list_int_column", Integer.class)) + .isEqualTo(Arrays.asList(1, 2, 3, 4, 5)), + () -> + assertThat(row.getList("list_text_column", String.class)) + .isEqualTo(Arrays.asList("apple", "banana", "cherry")), + () -> + assertThat(row.getList("list_of_sets_column", Set.class)) + .isEqualTo( + Arrays.asList( + new HashSet<>(Arrays.asList("apple", "banana")), + new HashSet<>(Arrays.asList("carrot", "spinach")))), + + // Maps + () -> + assertThat( + row.getMap("map_date_to_text_column", java.time.LocalDate.class, String.class)) + .isEqualTo( + Map.of( + java.time.LocalDate.parse("2025-01-27"), "event1", + java.time.LocalDate.parse("2025-02-01"), "event2")), + () -> + assertThat(row.getMap("map_text_to_int_column", String.class, Integer.class)) + .isEqualTo(Map.of("key1", 10, "key2", 20)), + () -> + assertThat(row.getMap("map_text_to_list_column", String.class, List.class)) + .isEqualTo( + Map.of( + "color", + Arrays.asList("red", "green"), + "fruit", + Arrays.asList("apple", "banana"))), + () -> + assertThat(row.getMap("map_text_to_set_column", String.class, Set.class)) + .isEqualTo( + Map.of( + "fruit", + new HashSet<>(Arrays.asList("apple", "banana")), + "vegetables", + new HashSet<>(Arrays.asList("carrot", "spinach")))), + + // Sets + () -> + assertThat(row.getSet("set_date_column", java.time.LocalDate.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + java.time.LocalDate.parse("2025-01-27"), + java.time.LocalDate.parse("2025-02-01")))), + () -> + assertThat(row.getSet("set_text_column", String.class)) + .isEqualTo(new HashSet<>(Arrays.asList("apple", "orange", "banana"))), + () -> + assertThat(row.getSet("set_of_maps_column", Map.class)) + .isEqualTo( + new HashSet<>( + Arrays.asList( + Map.of("key1", 10, "key2", 20), Map.of("keyA", 5, "keyB", 10)))), + + // Other Basic Types + () -> assertThat(row.getShort("smallint_column")).isEqualTo((short) 32767), + () -> assertThat(row.getInt("mediumint_column")).isEqualTo(8388607), + () -> assertThat(row.getInt("int_column")).isEqualTo(2147483647), + () -> assertThat(row.getString("enum_column")).isEqualTo("OptionA"), + () -> assertThat(row.getString("year_column")).isEqualTo("2025"), + () -> + assertThat(row.getString("longtext_column")) + .isEqualTo( + "Very long text data that exceeds the medium text column length for long text."), + () -> assertThat(row.getString("tinytext_column")).isEqualTo("Short text for tinytext."), + () -> + assertThat(row.getString("mediumtext_column")) + .isEqualTo("Longer text data for mediumtext column."), + () -> + assertThat(row.getString("text_column")) + .isEqualTo("This is some sample text data for the text column."), + () -> + assertThat(row.getLocalTime("time_column")) + .isEqualTo(java.time.LocalTime.parse("12:30:00.000000000")), + () -> + assertThat(row.getInstant("timestamp_column")) + .isEqualTo(java.time.Instant.parse("2025-01-27T10:30:00.123Z")), + () -> + assertThat(row.getBigInteger("varint_column")) + .isEqualTo(java.math.BigInteger.valueOf(123456789L))); } } diff --git a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java index 9323254360..16d54754f1 100644 --- a/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java +++ b/v2/spanner-to-sourcedb/src/test/java/com/google/cloud/teleport/v2/templates/SpannerToSourceDbITBase.java @@ -15,6 +15,7 @@ */ package com.google.cloud.teleport.v2.templates; +import static com.google.cloud.teleport.v2.spanner.migrations.constants.Constants.MYSQL_SOURCE_TYPE; import static org.apache.beam.it.truthmatchers.PipelineAsserts.assertThatPipeline; import com.google.cloud.teleport.v2.spanner.migrations.shard.Shard; @@ -34,6 +35,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.beam.it.common.PipelineLauncher; import org.apache.beam.it.common.utils.IORedirectUtil; @@ -207,7 +209,9 @@ public PipelineLauncher.LaunchInfo launchDataflowJob( put( "sourceShardsFilePath", getGcsPath( - sourceType != null ? "input/cassandra-config.conf" : "input/shard.json", + !Objects.equals(sourceType, MYSQL_SOURCE_TYPE) + ? "input/cassandra-config.conf" + : "input/shard.json", gcsResourceManager)); put("changeStreamName", "allstream"); put("dlqGcsPubSubSubscription", subscriptionName); diff --git a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql index 3fcc26b3bf..49830099eb 100644 --- a/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql +++ b/v2/spanner-to-sourcedb/src/test/resources/SpannerToCassandraSourceIT/cassandra-schema.sql @@ -4,6 +4,11 @@ CREATE TABLE users ( "from" text ); +CREATE TABLE users2 ( + id int PRIMARY KEY, + full_name text +); + CREATE TABLE AllDatatypeTransformation ( varchar_column text PRIMARY KEY, tinyint_column tinyint, From 5f6242ba216dd24941e016258d5144e2f6100cd1 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Mon, 10 Feb 2025 17:12:25 +0530 Subject: [PATCH 17/19] Splotless fixes --- .../google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java index 06e8f4f36c..40633485d1 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java @@ -129,7 +129,7 @@ public String convertJsonToPostgresInterval(String jsonValue, String columnName) try { ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(jsonValue); - + if (!rootNode.isObject() || !rootNode.has("months") || !rootNode.has("hours") From 716759c6163b2fcbc1b68689ed3be82192c3bac8 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Tue, 11 Feb 2025 09:02:15 +0530 Subject: [PATCH 18/19] Handle Retry Missing Exception Category --- .../teleport/v2/templates/transforms/SourceWriterFn.java | 4 ++++ 1 file changed, 4 insertions(+) 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 f7f848f389..08b823a899 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 @@ -15,6 +15,8 @@ */ package com.google.cloud.teleport.v2.templates.transforms; +import com.datastax.oss.driver.api.core.AllNodesFailedException; +import com.datastax.oss.driver.api.core.connection.ConnectionInitException; import com.datastax.oss.driver.api.core.servererrors.QueryExecutionException; import com.datastax.oss.driver.api.core.type.codec.CodecNotFoundException; import com.fasterxml.jackson.databind.DeserializationFeature; @@ -244,6 +246,8 @@ public void processElement(ProcessContext c) { | com.mysql.cj.jdbc.exceptions.CommunicationsException | java.sql.SQLIntegrityConstraintViolationException | java.sql.SQLTransientConnectionException + | AllNodesFailedException + | ConnectionInitException | ConnectionException ex) { outputWithTag(c, Constants.RETRYABLE_ERROR_TAG, ex.getMessage(), spannerRec); } catch (java.sql.SQLNonTransientConnectionException ex) { From 1dab953fbe44b7b7632779b2aa133ad628836659 Mon Sep 17 00:00:00 2001 From: pawankashyapollion Date: Tue, 11 Feb 2025 11:03:39 +0530 Subject: [PATCH 19/19] Rebase Issue fixes --- .../cloud/teleport/v2/utils/DatastreamToPostgresDML.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java index 40633485d1..e855dfff78 100644 --- a/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java +++ b/v2/datastream-to-sql/src/main/java/com/google/cloud/teleport/v2/utils/DatastreamToPostgresDML.java @@ -114,10 +114,6 @@ public String cleanDataTypeValueSql( if (dataType.startsWith("_")) { return convertJsonToPostgresArray(columnValue, dataType.toUpperCase(), columnName); } - // Arrays in Postgres are prefixed with underscore e.g. _INT4 for integer array. - if (dataType.startsWith("_")) { - return convertJsonToPostgresArray(columnValue); - } return columnValue; }