Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* Copyright (C) 2026 Google LLC
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package com.google.cloud.teleport.v2.templates;

import static com.google.cloud.teleport.v2.templates.constants.DatastreamToSpannerConstants.CONVERSION_ERRORS_COUNTER_NAME;

import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.cloud.teleport.v2.spanner.ddl.Ddl;
import com.google.cloud.teleport.v2.spanner.migrations.convertors.ChangeEventSpannerConvertor;
import com.google.cloud.teleport.v2.templates.constants.DatastreamToSpannerConstants;
import com.google.cloud.teleport.v2.templates.datastream.ChangeEventConvertor;
import com.google.cloud.teleport.v2.templates.datastream.DatastreamConstants;
import com.google.cloud.teleport.v2.values.FailsafeElement;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollectionView;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class CreateKeyValuePairsWithPrimaryKeyHashDoFn
extends DoFn<FailsafeElement<String, String>, KV<Long, FailsafeElement<String, String>>> {

private static final Logger LOG =
LoggerFactory.getLogger(CreateKeyValuePairsWithPrimaryKeyHashDoFn.class);

private final PCollectionView<Ddl> ddlView;

// Jackson Object mapper.
private transient ObjectMapper mapper;

private final Counter conversionErrors =
Metrics.counter(SpannerTransactionWriterDoFn.class, CONVERSION_ERRORS_COUNTER_NAME);

public CreateKeyValuePairsWithPrimaryKeyHashDoFn(PCollectionView<Ddl> ddlView) {
this.ddlView = ddlView;
}

@Setup
public void setup() {
mapper = new ObjectMapper();
mapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);
}

@ProcessElement
public void processElement(ProcessContext c) {
FailsafeElement<String, String> msg = c.element();
String tableName = "";
try {
// msg.getPayload() contains transformed change event and hence msg.getPayload() should be
// used and not msg.getOriginalPayload()
JsonNode changeEvent = mapper.readTree(msg.getPayload());
Ddl ddl = c.sideInput(ddlView);

tableName = changeEvent.get(DatastreamConstants.EVENT_TABLE_NAME_KEY).asText();
ChangeEventConvertor.convertChangeEventColumnKeysToLowerCase(changeEvent);
ChangeEventConvertor.verifySpannerSchema(ddl, changeEvent);
com.google.cloud.spanner.Key primaryKey =
ChangeEventSpannerConvertor.changeEventToPrimaryKey(
tableName, ddl, changeEvent, /* convertNameToLowerCase= */ true);
String finalKeyString = tableName + "_" + primaryKey.toString();
Long finalKey = (long) finalKeyString.hashCode();
c.output(KV.of(finalKey, msg));
} catch (Exception e) {
LOG.error(
"Error while converting change event to primary key hash for tableName=" + tableName, e);
// Errors that result during Event conversions are not retryable.
// Making a copy, as the input must not be mutated.
FailsafeElement<String, String> output = FailsafeElement.of(msg);
output.setErrorMessage(e.getMessage());
c.output(DatastreamToSpannerConstants.PERMANENT_ERROR_TAG, output);
conversionErrors.inc();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,18 +17,26 @@

import com.google.auto.value.AutoValue;
import com.google.cloud.Timestamp;
import com.google.cloud.teleport.v2.coders.FailsafeElementCoder;
import com.google.cloud.teleport.v2.spanner.ddl.Ddl;
import com.google.cloud.teleport.v2.templates.constants.DatastreamToSpannerConstants;
import com.google.cloud.teleport.v2.values.FailsafeElement;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
import org.apache.beam.sdk.transforms.Flatten;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.PInput;
Expand Down Expand Up @@ -93,29 +101,53 @@ public SpannerTransactionWriter(
@Override
public SpannerTransactionWriter.Result expand(
PCollection<FailsafeElement<String, String>> input) {
PCollectionTuple spannerWriteResults =
PCollectionTuple keyedEvents =
input.apply(
"Write Mutations",
ParDo.of(
new SpannerTransactionWriterDoFn(
spannerConfig,
shadowTableSpannerConfig,
ddlView,
shadowTableDdlView,
shadowTablePrefix,
sourceType,
isRegularRunMode))
.withSideInputs(ddlView, shadowTableDdlView)
"Key By PK Hash",
ParDo.of(new CreateKeyValuePairsWithPrimaryKeyHashDoFn(ddlView))
.withSideInputs(ddlView)
.withOutputTags(
DatastreamToSpannerConstants.SUCCESSFUL_EVENT_TAG,
TupleTagList.of(
Arrays.asList(
DatastreamToSpannerConstants.PERMANENT_ERROR_TAG,
DatastreamToSpannerConstants.RETRYABLE_ERROR_TAG))));
DatastreamToSpannerConstants.SUCCESSFUL_KEYED_EVENT_TAG,
TupleTagList.of(List.of(DatastreamToSpannerConstants.PERMANENT_ERROR_TAG))));
PCollectionTuple spannerWriteResults =
keyedEvents
.get(DatastreamToSpannerConstants.SUCCESSFUL_KEYED_EVENT_TAG)
.setCoder(
KvCoder.of(
VarLongCoder.of(),
FailsafeElementCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())))
.apply("Reshuffle Keyed Events", Reshuffle.of())
.apply(
"Write Mutations",
ParDo.of(
new SpannerTransactionWriterDoFn(
spannerConfig,
shadowTableSpannerConfig,
ddlView,
shadowTableDdlView,
shadowTablePrefix,
sourceType,
isRegularRunMode))
.withSideInputs(ddlView, shadowTableDdlView)
.withOutputTags(
DatastreamToSpannerConstants.SUCCESSFUL_EVENT_TAG,
TupleTagList.of(
Arrays.asList(
DatastreamToSpannerConstants.PERMANENT_ERROR_TAG,
DatastreamToSpannerConstants.RETRYABLE_ERROR_TAG))));

PCollection<FailsafeElement<String, String>> keyedEventsErrorRecords =
keyedEvents.get(DatastreamToSpannerConstants.PERMANENT_ERROR_TAG);

PCollection<FailsafeElement<String, String>> permanentErrorRecords =
PCollectionList.of(
spannerWriteResults.get(DatastreamToSpannerConstants.PERMANENT_ERROR_TAG))
.and(keyedEventsErrorRecords)
.apply(Flatten.pCollections());

return Result.create(
spannerWriteResults.get(DatastreamToSpannerConstants.SUCCESSFUL_EVENT_TAG),
spannerWriteResults.get(DatastreamToSpannerConstants.PERMANENT_ERROR_TAG),
permanentErrorRecords,
spannerWriteResults.get(DatastreamToSpannerConstants.RETRYABLE_ERROR_TAG));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.TupleTag;
import org.joda.time.Duration;
Expand All @@ -77,8 +78,8 @@
* <p>Change events that failed to be written will be pushed onto the secondary output tagged with
* PERMANENT_ERROR_TAG/RETRYABLE_ERROR_TAG along with the exception that caused the failure.
*/
class SpannerTransactionWriterDoFn extends DoFn<FailsafeElement<String, String>, Timestamp>
implements Serializable {
class SpannerTransactionWriterDoFn
extends DoFn<KV<Long, FailsafeElement<String, String>>, Timestamp> implements Serializable {

// TODO - Change Cloud Spanner nomenclature in code used to read DDL.

Expand Down Expand Up @@ -239,7 +240,7 @@ public void teardown() {

@ProcessElement
public void processElement(ProcessContext c) {
FailsafeElement<String, String> msg = c.element();
FailsafeElement<String, String> msg = c.element().getValue();
Ddl ddl = c.sideInput(ddlView);
// TODO: pass shadow table ddl to shdaow tble mutaiton generator and sequence reader.
Ddl shadowTableDdl = c.sideInput(shadowTableDdlView);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import com.google.cloud.Timestamp;
import com.google.cloud.teleport.v2.values.FailsafeElement;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.TupleTag;

/** Class to maintain all the constants used in the pipeline. */
Expand All @@ -42,6 +43,10 @@ public class DatastreamToSpannerConstants {
/* The Tag for Successful mutations. */
public static final TupleTag<Timestamp> SUCCESSFUL_EVENT_TAG = new TupleTag<Timestamp>() {};

/* The tag for successfully keyed events. */
public static final TupleTag<KV<Long, FailsafeElement<String, String>>>
SUCCESSFUL_KEYED_EVENT_TAG = new TupleTag<KV<Long, FailsafeElement<String, String>>>() {};

/* Max DoFns per dataflow worker in a streaming pipeline. */
public static final int MAX_DOFN_PER_WORKER = 500;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public class ChangeEventConvertor {

private ChangeEventConvertor() {}

static void verifySpannerSchema(Ddl ddl, JsonNode changeEvent)
public static void verifySpannerSchema(Ddl ddl, JsonNode changeEvent)
throws ChangeEventConvertorException, InvalidChangeEventException, DroppedTableException {
String tableName = changeEvent.get(DatastreamConstants.EVENT_TABLE_NAME_KEY).asText();
if (ddl.table(tableName) == null) {
Expand Down Expand Up @@ -66,7 +66,7 @@ static void verifySpannerSchema(Ddl ddl, JsonNode changeEvent)
}
}

static void convertChangeEventColumnKeysToLowerCase(JsonNode changeEvent)
public static void convertChangeEventColumnKeysToLowerCase(JsonNode changeEvent)
throws ChangeEventConvertorException, InvalidChangeEventException {
List<String> changeEventKeys = ChangeEventUtils.getEventColumnKeys(changeEvent);
ObjectNode jsonNode = (ObjectNode) changeEvent;
Expand Down
Loading
Loading