-
Notifications
You must be signed in to change notification settings - Fork 1.1k
Basic Changes to use Upstream Cassandra io for Bulk Cassandra Migration #2129
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
VardhanThigle
merged 2 commits into
GoogleCloudPlatform:main
from
VardhanThigle:existing-cassandra-io-basic
Jan 14, 2025
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
29 changes: 29 additions & 0 deletions
29
.../com/google/cloud/teleport/v2/source/reader/io/cassandra/iowrapper/CassandraDefaults.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| /* | ||
| * 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.source.reader.io.cassandra.iowrapper; | ||
|
|
||
| import org.apache.beam.sdk.util.FluentBackoff; | ||
| import org.joda.time.Duration; | ||
|
|
||
| public class CassandraDefaults { | ||
|
|
||
| /** Fluent Backoff for Cassandra Schema Discovery. */ | ||
| public static final FluentBackoff DEFAULT_CASSANDRA_SCHEMA_DISCOVERY_BACKOFF = | ||
| FluentBackoff.DEFAULT.withMaxCumulativeBackoff(Duration.standardMinutes(5L)); | ||
|
|
||
| private CassandraDefaults() {} | ||
| ; | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
139 changes: 139 additions & 0 deletions
139
...ogle/cloud/teleport/v2/source/reader/io/cassandra/iowrapper/CassandraIOWrapperHelper.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,139 @@ | ||
| /* | ||
| * 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.source.reader.io.cassandra.iowrapper; | ||
|
|
||
| import static com.google.cloud.teleport.v2.source.reader.io.cassandra.iowrapper.CassandraDefaults.DEFAULT_CASSANDRA_SCHEMA_DISCOVERY_BACKOFF; | ||
|
|
||
| import com.google.cloud.teleport.v2.source.reader.io.cassandra.schema.CassandraSchemaDiscovery; | ||
| import com.google.cloud.teleport.v2.source.reader.io.datasource.DataSource; | ||
| import com.google.cloud.teleport.v2.source.reader.io.exception.SchemaDiscoveryException; | ||
| import com.google.cloud.teleport.v2.source.reader.io.row.SourceRow; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SchemaDiscovery; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SchemaDiscoveryImpl; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceSchema; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceSchemaReference; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceTableReference; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceTableSchema; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.typemapping.UnifiedTypeMapper.MapperType; | ||
| import com.google.cloud.teleport.v2.spanner.migrations.schema.SourceColumnType; | ||
| import com.google.common.collect.ImmutableList; | ||
| import com.google.common.collect.ImmutableMap; | ||
| import java.io.FileNotFoundException; | ||
| import java.util.List; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.values.PBegin; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| /** Static Utility Class to provide basic functionality to {@link CassandraIoWrapper}. */ | ||
| class CassandraIOWrapperHelper { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(CassandraIOWrapperHelper.class); | ||
|
|
||
| static DataSource buildDataSource(String gcsPath) { | ||
| DataSource dataSource; | ||
| try { | ||
| dataSource = | ||
| DataSource.ofCassandra( | ||
| CassandraDataSource.builder().setOptionsMapFromGcsFile(gcsPath).build()); | ||
| } catch (FileNotFoundException e) { | ||
| LOG.error("Unable to find driver config file in {}. Cause ", gcsPath, e); | ||
| throw (new SchemaDiscoveryException(e)); | ||
| } | ||
| return dataSource; | ||
| } | ||
|
|
||
| static SchemaDiscovery buildSchemaDiscovery() { | ||
| return new SchemaDiscoveryImpl( | ||
| new CassandraSchemaDiscovery(), DEFAULT_CASSANDRA_SCHEMA_DISCOVERY_BACKOFF); | ||
| } | ||
|
|
||
| static ImmutableList<String> getTablesToRead( | ||
| List<String> sourceTables, | ||
| DataSource dataSource, | ||
| SchemaDiscovery schemaDiscovery, | ||
| SourceSchemaReference sourceSchemaReference) { | ||
| ImmutableList<String> tablesToRead; | ||
| if (sourceTables.isEmpty()) { | ||
| tablesToRead = schemaDiscovery.discoverTables(dataSource, sourceSchemaReference); | ||
| LOG.info("Auto Discovered SourceTables = {}, Tables = {}", sourceTables, tablesToRead); | ||
| } else { | ||
| tablesToRead = ImmutableList.copyOf(sourceTables); | ||
| LOG.info("Using passed SourceTables = {}", sourceTables); | ||
| } | ||
| return tablesToRead; | ||
| } | ||
|
|
||
| static SourceSchema getSourceSchema( | ||
| SchemaDiscovery schemaDiscovery, | ||
| DataSource dataSource, | ||
| SourceSchemaReference sourceSchemaReference, | ||
| ImmutableList<String> tables) { | ||
|
|
||
| SourceSchema.Builder sourceSchemaBuilder = | ||
| SourceSchema.builder().setSchemaReference(sourceSchemaReference); | ||
| ImmutableMap<String, ImmutableMap<String, SourceColumnType>> tableSchemas = | ||
| schemaDiscovery.discoverTableSchema(dataSource, sourceSchemaReference, tables); | ||
| LOG.info("Found table schemas: {}", tableSchemas); | ||
| tableSchemas.entrySet().stream() | ||
| .map( | ||
| tableEntry -> { | ||
| SourceTableSchema.Builder sourceTableSchemaBuilder = | ||
| SourceTableSchema.builder(MapperType.CASSANDRA).setTableName(tableEntry.getKey()); | ||
| tableEntry | ||
| .getValue() | ||
| .entrySet() | ||
| .forEach( | ||
| colEntry -> | ||
| sourceTableSchemaBuilder.addSourceColumnNameToSourceColumnType( | ||
| colEntry.getKey(), colEntry.getValue())); | ||
| return sourceTableSchemaBuilder.build(); | ||
| }) | ||
| .forEach(sourceSchemaBuilder::addTableSchema); | ||
| return sourceSchemaBuilder.build(); | ||
| } | ||
|
|
||
| static ImmutableMap<SourceTableReference, PTransform<PBegin, PCollection<SourceRow>>> | ||
| getTableReaders(DataSource dataSource, SourceSchema sourceSchema) { | ||
| /* | ||
| * TODO(vardhanvthigle): Plugin alternate implementation if needed. | ||
| */ | ||
| CassandraTableReaderFactory cassandraTableReaderFactory = | ||
| new CassandraTableReaderFactoryCassandraIoImpl(); | ||
| ImmutableMap.Builder<SourceTableReference, PTransform<PBegin, PCollection<SourceRow>>> | ||
| tableReadersBuilder = ImmutableMap.builder(); | ||
| SourceSchemaReference sourceSchemaReference = sourceSchema.schemaReference(); | ||
| sourceSchema | ||
| .tableSchemas() | ||
| .forEach( | ||
| tableSchema -> { | ||
| SourceTableReference sourceTableReference = | ||
| SourceTableReference.builder() | ||
| .setSourceSchemaReference(sourceSchemaReference) | ||
| .setSourceTableSchemaUUID(tableSchema.tableSchemaUUID()) | ||
| .setSourceTableName(tableSchema.tableName()) | ||
| .build(); | ||
| var tableReader = | ||
| cassandraTableReaderFactory.getTableReader( | ||
| dataSource.cassandra(), sourceSchemaReference, tableSchema); | ||
| tableReadersBuilder.put(sourceTableReference, tableReader); | ||
| }); | ||
| return tableReadersBuilder.build(); | ||
| } | ||
|
|
||
| private CassandraIOWrapperHelper() {} | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
39 changes: 39 additions & 0 deletions
39
...e/cloud/teleport/v2/source/reader/io/cassandra/iowrapper/CassandraTableReaderFactory.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,39 @@ | ||
| /* | ||
| * 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.source.reader.io.cassandra.iowrapper; | ||
|
|
||
| import com.google.cloud.teleport.v2.source.reader.io.row.SourceRow; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceSchemaReference; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceTableSchema; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.values.PBegin; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
|
|
||
| public interface CassandraTableReaderFactory { | ||
|
|
||
| /** | ||
| * Returns a Table Reader for given Cassandra Source. | ||
| * | ||
| * @param cassandraDataSource | ||
| * @param sourceSchemaReference | ||
| * @param sourceTableSchema | ||
| * @return table reader for the source. | ||
| */ | ||
| PTransform<PBegin, PCollection<SourceRow>> getTableReader( | ||
| CassandraDataSource cassandraDataSource, | ||
| SourceSchemaReference sourceSchemaReference, | ||
| SourceTableSchema sourceTableSchema); | ||
| } |
98 changes: 98 additions & 0 deletions
98
...t/v2/source/reader/io/cassandra/iowrapper/CassandraTableReaderFactoryCassandraIoImpl.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,98 @@ | ||
| /* | ||
| * 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.source.reader.io.cassandra.iowrapper; | ||
|
|
||
| import com.datastax.oss.driver.api.core.config.DriverExecutionProfile; | ||
| import com.datastax.oss.driver.api.core.config.TypedDriverOption; | ||
| import com.google.cloud.teleport.v2.source.reader.io.cassandra.rowmapper.CassandraSourceRowMapper; | ||
| import com.google.cloud.teleport.v2.source.reader.io.cassandra.rowmapper.CassandraSourceRowMapperFactoryFn; | ||
| import com.google.cloud.teleport.v2.source.reader.io.row.SourceRow; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceSchemaReference; | ||
| import com.google.cloud.teleport.v2.source.reader.io.schema.SourceTableSchema; | ||
| import com.google.common.annotations.VisibleForTesting; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.beam.sdk.coders.SerializableCoder; | ||
| import org.apache.beam.sdk.io.cassandra.CassandraIO; | ||
| import org.apache.beam.sdk.io.cassandra.CassandraIO.Read; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.values.PBegin; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
|
|
||
| /** | ||
| * Generate Table Reader For Cassandra using the upstream {@link CassandraIO.Read} implementation. | ||
| */ | ||
| public class CassandraTableReaderFactoryCassandraIoImpl implements CassandraTableReaderFactory { | ||
|
|
||
| /** | ||
| * Returns a Table Reader for given Cassandra Source using the upstream {@link CassandraIO.Read}. | ||
| * | ||
| * @param cassandraDataSource | ||
| * @param sourceSchemaReference | ||
| * @param sourceTableSchema | ||
| * @return table reader for the source. | ||
| */ | ||
| @Override | ||
| public PTransform<PBegin, PCollection<SourceRow>> getTableReader( | ||
| CassandraDataSource cassandraDataSource, | ||
| SourceSchemaReference sourceSchemaReference, | ||
| SourceTableSchema sourceTableSchema) { | ||
| CassandraSourceRowMapper cassandraSourceRowMapper = | ||
| getSourceRowMapper(sourceSchemaReference, sourceTableSchema); | ||
| DriverExecutionProfile profile = | ||
| cassandraDataSource.driverConfigLoader().getInitialConfig().getDefaultProfile(); | ||
| final Read<SourceRow> tableReader = | ||
| CassandraIO.<SourceRow>read() | ||
| .withTable(sourceTableSchema.tableName()) | ||
| .withHosts( | ||
| cassandraDataSource.contactPoints().stream() | ||
| .map(p -> p.getHostString()) | ||
| .collect(Collectors.toList())) | ||
| .withPort(cassandraDataSource.contactPoints().get(0).getPort()) | ||
| .withKeyspace(cassandraDataSource.loggedKeySpace()) | ||
| .withLocalDc(cassandraDataSource.localDataCenter()) | ||
| .withConsistencyLevel( | ||
| profile.getString(TypedDriverOption.REQUEST_SERIAL_CONSISTENCY.getRawOption())) | ||
| .withEntity(SourceRow.class) | ||
| .withCoder(SerializableCoder.of(SourceRow.class)) | ||
| .withMapperFactoryFn( | ||
| CassandraSourceRowMapperFactoryFn.create(cassandraSourceRowMapper)); | ||
| return setCredentials(tableReader, profile); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| protected CassandraIO.Read<SourceRow> setCredentials( | ||
| CassandraIO.Read<SourceRow> tableReader, DriverExecutionProfile profile) { | ||
| if (profile.isDefined(TypedDriverOption.AUTH_PROVIDER_USER_NAME.getRawOption())) { | ||
| tableReader = | ||
| tableReader.withUsername( | ||
| profile.getString(TypedDriverOption.AUTH_PROVIDER_USER_NAME.getRawOption())); | ||
| } | ||
| if (profile.isDefined(TypedDriverOption.AUTH_PROVIDER_PASSWORD.getRawOption())) { | ||
| tableReader = | ||
| tableReader.withPassword( | ||
| profile.getString(TypedDriverOption.AUTH_PROVIDER_PASSWORD.getRawOption())); | ||
| } | ||
| return tableReader; | ||
| } | ||
|
|
||
| private CassandraSourceRowMapper getSourceRowMapper( | ||
| SourceSchemaReference sourceSchemaReference, SourceTableSchema sourceTableSchema) { | ||
| return CassandraSourceRowMapper.builder() | ||
| .setSourceTableSchema(sourceTableSchema) | ||
| .setSourceSchemaReference(sourceSchemaReference) | ||
| .build(); | ||
| } | ||
| } | ||
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.