-
Notifications
You must be signed in to change notification settings - Fork 1.1k
Feat: Add GCS sink to bulk migration pipeline #3139
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
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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 java.util.Objects; | ||
| import org.apache.beam.sdk.coders.DefaultCoder; | ||
| import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; | ||
|
|
||
| @DefaultCoder(AvroCoder.class) | ||
| public class AvroDestination { | ||
| public String name; | ||
| public String jsonSchema; | ||
|
|
||
| // Needed for serialization | ||
| public AvroDestination() {} | ||
|
|
||
| public AvroDestination(String name, String jsonSchema) { | ||
| this.name = name; | ||
| this.jsonSchema = jsonSchema; | ||
| } | ||
|
|
||
| public static AvroDestination of(String name, String jsonSchema) { | ||
| return new AvroDestination(name, jsonSchema); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) { | ||
| return true; | ||
| } | ||
| if (!(o instanceof AvroDestination)) { | ||
| return false; | ||
| } | ||
| AvroDestination that = (AvroDestination) o; | ||
| return Objects.equals(name, that.name) && Objects.equals(jsonSchema, that.jsonSchema); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(name, jsonSchema); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,22 +31,38 @@ | |
| import java.util.Map; | ||
| import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; | ||
| import org.apache.beam.sdk.coders.SerializableCoder; | ||
| import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; | ||
| import org.apache.beam.sdk.extensions.avro.io.AvroIO; | ||
| import org.apache.beam.sdk.io.Compression; | ||
| import org.apache.beam.sdk.io.FileIO; | ||
| import org.apache.beam.sdk.io.FileIO.Write.FileNaming; | ||
| import org.apache.beam.sdk.io.FileSystems; | ||
| import org.apache.beam.sdk.io.WriteFilesResult; | ||
| import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; | ||
| import org.apache.beam.sdk.io.gcp.spanner.MutationGroup; | ||
| import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig; | ||
| import org.apache.beam.sdk.io.gcp.spanner.SpannerWriteResult; | ||
| import org.apache.beam.sdk.metrics.Metrics; | ||
| import org.apache.beam.sdk.transforms.Contextful; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.transforms.SerializableFunction; | ||
| import org.apache.beam.sdk.transforms.windowing.BoundedWindow; | ||
| import org.apache.beam.sdk.transforms.windowing.PaneInfo; | ||
| import org.apache.beam.sdk.values.PBegin; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.PCollectionTuple; | ||
| import org.apache.beam.sdk.values.TupleTagList; | ||
| import org.apache.commons.codec.digest.DigestUtils; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| public class MigrateTableTransform extends PTransform<PBegin, PCollection<Void>> { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(MigrateTableTransform.class); | ||
| public static final String GCS_RECORDS_WRITTEN = "gcs_records_written"; | ||
|
|
||
| private SourceDbToSpannerOptions options; | ||
| private transient SourceDbToSpannerOptions options; | ||
manitgupta marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| private SpannerConfig spannerConfig; | ||
| private Ddl ddl; | ||
| private ISchemaMapper schemaMapper; | ||
|
|
@@ -81,6 +97,19 @@ public PCollection<Void> expand(PBegin input) { | |
| PCollectionTuple rowsAndTables = input.apply("Read_rows", readerTransform.readTransform()); | ||
| PCollection<SourceRow> sourceRows = rowsAndTables.get(readerTransform.sourceRowTag()); | ||
|
|
||
| if (options.getGcsOutputDirectory() != null && !options.getGcsOutputDirectory().isEmpty()) { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. thinking out loud, should this be behind a enable data validation flag instead of GCS DIR flag?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am open to it, although GCS sink functionality is a more general one, with a usage in validation. We can rename to something like |
||
| String avroDirectory; | ||
| if (shardId.isEmpty()) { | ||
| avroDirectory = options.getGcsOutputDirectory(); | ||
| } else { | ||
| avroDirectory = | ||
| FileSystems.matchNewResource(options.getGcsOutputDirectory(), true) | ||
| .resolve(shardId, StandardResolveOptions.RESOLVE_DIRECTORY) | ||
|
Comment on lines
+106
to
+107
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what are the pros/cons of choosing this over gcs_dir/table/shardId? are there other folder structures considered?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think there is a large advantage of one over the other, and this change-able if we see most customers asking for the other. |
||
| .toString(); | ||
| } | ||
| writeToGCS(sourceRows, avroDirectory); | ||
| } | ||
|
|
||
| CustomTransformation customTransformation = | ||
| CustomTransformation.builder( | ||
| options.getTransformationJarPath(), options.getTransformationClassName()) | ||
|
|
@@ -143,4 +172,56 @@ public PCollection<Void> expand(PBegin input) { | |
| .setCoder(SerializableCoder.of(RowContext.class))); | ||
| return spannerWriteResult.getOutput(); | ||
| } | ||
|
|
||
| public WriteFilesResult<AvroDestination> writeToGCS( | ||
| PCollection<SourceRow> sourceRows, String gcsOutputDirectory) { | ||
| String shardIdForMetric = this.shardId; | ||
| String metricName = | ||
| StringUtils.isEmpty(shardIdForMetric) | ||
| ? GCS_RECORDS_WRITTEN | ||
| : String.join("_", GCS_RECORDS_WRITTEN, shardIdForMetric); | ||
| return sourceRows.apply( | ||
| "WriteAvroToGCS", | ||
| FileIO.<AvroDestination, SourceRow>writeDynamic() | ||
| .by( | ||
| (record) -> | ||
| AvroDestination.of( | ||
| record.tableName(), record.getPayload().getSchema().toString())) | ||
| .via( | ||
| Contextful.fn( | ||
| record -> { | ||
| Metrics.counter(MigrateTableTransform.class, metricName).inc(); | ||
manitgupta marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return record.getPayload(); | ||
| }), | ||
| Contextful.fn(destination -> AvroIO.sink(destination.jsonSchema))) | ||
| .withDestinationCoder(AvroCoder.of(AvroDestination.class)) | ||
| .to(gcsOutputDirectory) | ||
| .withNumShards(0) | ||
| .withNaming((SerializableFunction<AvroDestination, FileNaming>) AvroFileNaming::new)); | ||
| } | ||
|
|
||
| static class AvroFileNaming implements FileIO.Write.FileNaming { | ||
|
|
||
| private final FileIO.Write.FileNaming defaultNaming; | ||
| private final AvroDestination avroDestination; | ||
|
|
||
| public AvroFileNaming(AvroDestination avroDestination) { | ||
| defaultNaming = | ||
| FileIO.Write.defaultNaming(DigestUtils.md5Hex(avroDestination.jsonSchema), ".avro"); | ||
manitgupta marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| this.avroDestination = avroDestination; | ||
| } | ||
|
|
||
| @Override | ||
| public String getFilename( | ||
| BoundedWindow window, | ||
| PaneInfo pane, | ||
| int numShards, | ||
| int shardIndex, | ||
| Compression compression) { | ||
| String subDir = avroDestination.name; | ||
| return subDir | ||
| + "/" | ||
| + defaultNaming.getFilename(window, pane, numShards, shardIndex, compression); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,49 @@ | ||
| /* | ||
| * 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.junit.Assert.assertEquals; | ||
| import static org.junit.Assert.assertNotEquals; | ||
|
|
||
| import org.junit.Test; | ||
| import org.junit.runner.RunWith; | ||
| import org.junit.runners.JUnit4; | ||
|
|
||
| /** Unit tests for {@link AvroDestination}. */ | ||
| @RunWith(JUnit4.class) | ||
| public class AvroDestinationTest { | ||
|
|
||
| @Test | ||
| public void testEqualsAndHashCode() { | ||
| AvroDestination dest1 = AvroDestination.of("table1", "schema1"); | ||
| AvroDestination dest2 = AvroDestination.of("table1", "schema1"); | ||
| AvroDestination dest3 = AvroDestination.of("table2", "schema1"); | ||
| AvroDestination dest4 = AvroDestination.of("table1", "schema2"); | ||
|
|
||
| assertEquals(dest1, dest2); | ||
| assertEquals(dest1.hashCode(), dest2.hashCode()); | ||
| assertNotEquals(dest1, dest3); | ||
| assertNotEquals(dest1, dest4); | ||
| assertNotEquals(dest3, dest4); | ||
| } | ||
|
|
||
| @Test | ||
| public void testOf() { | ||
| AvroDestination destination = AvroDestination.of("tableName", "jsonSchema"); | ||
| assertEquals("tableName", destination.name); | ||
| assertEquals("jsonSchema", destination.jsonSchema); | ||
| } | ||
| } |
Uh oh!
There was an error while loading. Please reload this page.