Skip to content

Commit

Permalink
refactor: add fields, refactor initialization, and add test (#11)
Browse files Browse the repository at this point in the history
  • Loading branch information
zoercai authored Jun 18, 2021
1 parent e7f18ff commit 0c9d0e2
Show file tree
Hide file tree
Showing 15 changed files with 469 additions and 241 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static java.util.stream.Collectors.toList;
import static org.apache.beam.sdk.io.gcp.spanner.MutationUtils.isPointDelete;
import static org.apache.beam.sdk.io.gcp.spanner.cdc.NameGenerator.generateMetadataTableName;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;

Expand Down Expand Up @@ -56,8 +57,8 @@
import org.apache.beam.sdk.io.gcp.spanner.cdc.DetectNewPartitions;
import org.apache.beam.sdk.io.gcp.spanner.cdc.PipelineInitializer;
import org.apache.beam.sdk.io.gcp.spanner.cdc.ReadPartitionChangeStream;
import org.apache.beam.sdk.io.gcp.spanner.cdc.model.DataChangesRecord;
import org.apache.beam.sdk.io.gcp.spanner.cdc.dao.PartitionMetadataDao;
import org.apache.beam.sdk.io.gcp.spanner.cdc.model.DataChangesRecord;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Metrics;
Expand Down Expand Up @@ -1381,32 +1382,28 @@ public PCollection<DataChangesRecord> expand(PBegin input) {
"SpannerIO.readChangeStream() requires the start time to be set.");

// Start time must be before end time
if (getExclusiveEndAt() != null && getInclusiveStartAt().toSqlTimestamp()
.after(getExclusiveEndAt().toSqlTimestamp())) {
if (getExclusiveEndAt() != null
&& getInclusiveStartAt().toSqlTimestamp().after(getExclusiveEndAt().toSqlTimestamp())) {
throw new IllegalArgumentException("Start time cannot be after end time.");
}

SpannerAccessor spannerAccessor = SpannerAccessor.getOrCreate(getSpannerConfig());
DatabaseAdminClient databaseAdminClient = spannerAccessor.getDatabaseAdminClient();
DatabaseClient databaseClient = spannerAccessor.getDatabaseClient();
Database changeStreamsDb = databaseAdminClient.getDatabase(getSpannerConfig().getInstanceId().get(),
getSpannerConfig().getDatabaseId().get());

// Start time must be within data retention period
// TODO: spanner dependency version is too old, need to be updated to see data retention period

// Start time must be after change stream creation time
checkArgument(!getInclusiveStartAt().toSqlTimestamp()
.before(changeStreamsDb.getCreateTime().toSqlTimestamp()),
"Start time must not be before the change stream creation time.");

PipelineInitializer pipelineInitializer = new PipelineInitializer();
PartitionMetadataDao partitionMetadataDao = new PartitionMetadataDao(databaseClient);
DatabaseId databaseId = DatabaseId.of(
getSpannerConfig().getProjectId().get(),
getSpannerConfig().getInstanceId().get(),
getSpannerConfig().getDatabaseId().get());
pipelineInitializer.initialize(
Database changeStreamsDb =
databaseAdminClient.getDatabase(
getSpannerConfig().getInstanceId().get(), getSpannerConfig().getDatabaseId().get());

DatabaseId databaseId =
DatabaseId.of(
getSpannerConfig().getProjectId().get(),
getSpannerConfig().getInstanceId().get(),
getSpannerConfig().getDatabaseId().get());
String partitionMetadataTableName = generateMetadataTableName(databaseId.getDatabase());

PartitionMetadataDao partitionMetadataDao =
new PartitionMetadataDao(databaseClient, partitionMetadataTableName);
PipelineInitializer.initialize(
databaseAdminClient,
partitionMetadataDao,
databaseId,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.beam.sdk.io.gcp.spanner.cdc;

import java.util.UUID;

public class NameGenerator {

private static final String METADATA_TABLE_NAME_FORMAT = "CDC_Partitions_%s_%s";

public static String generateMetadataTableName(String databaseId) {
// Maximum Spanner table name length is 128 characters.
// There are 16 characters in the name format.
// Maximum Spanner database ID length is 30 characters.
// UUID always generates a String with 36 characters.
// 128 - (16 + 30 + 36) = 46 characters short of the limit
return String.format(METADATA_TABLE_NAME_FORMAT, databaseId, UUID.randomUUID())
.replaceAll("-", "_");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,47 +34,59 @@
import com.google.cloud.spanner.DatabaseId;
import com.google.cloud.spanner.SpannerException;
import com.google.cloud.spanner.SpannerExceptionFactory;
import com.google.common.collect.ImmutableList;
import com.google.cloud.spanner.Value;
import com.google.spanner.admin.database.v1.UpdateDatabaseDdlMetadata;
import java.util.Collections;
import java.util.UUID;
import java.util.concurrent.ExecutionException;
import javax.annotation.Nullable;
import org.apache.beam.sdk.io.gcp.spanner.cdc.dao.PartitionMetadataDao;
import org.apache.beam.sdk.io.gcp.spanner.cdc.model.PartitionMetadata;
import org.apache.beam.sdk.io.gcp.spanner.cdc.model.PartitionMetadata.State;
import org.apache.beam.sdk.io.gcp.spanner.cdc.dao.PartitionMetadataDao;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;

public class PipelineInitializer {

private static final String DEFAULT_PARENT_PARTITION_TOKEN = "Parent0";
private static final ImmutableList<String> DEFAULT_PARENT_TOKENS = ImmutableList.of();
private static final long DEFAULT_HEARTBEAT_SECONDS = 1;

public void initialize(DatabaseAdminClient databaseAdminClient,
PartitionMetadataDao partitionMetadataDao, DatabaseId id, Timestamp inclusiveStartAt,
public static void initialize(
DatabaseAdminClient databaseAdminClient,
PartitionMetadataDao partitionMetadataDao,
DatabaseId id,
Timestamp inclusiveStartAt,
@Nullable Timestamp exclusiveEndAt) {
createMetadataTable(databaseAdminClient, id);
createFakeParentPartition(partitionMetadataDao, id, inclusiveStartAt, exclusiveEndAt);
createMetadataTable(databaseAdminClient, id, partitionMetadataDao.getTableName());
createFakeParentPartition(partitionMetadataDao, inclusiveStartAt, exclusiveEndAt);
}

private void createMetadataTable(DatabaseAdminClient databaseAdminClient, DatabaseId id) {
private static void createMetadataTable(
DatabaseAdminClient databaseAdminClient, DatabaseId id, String tableName) {
final String metadataCreateStmt =
"CREATE TABLE CDC_Partitions_"
+ id.getName()
+ "_"
+ UUID.randomUUID()
"CREATE TABLE "
+ tableName
+ " ("
+ COLUMN_PARTITION_TOKEN + " STRING(MAX) NOT NULL,"
+ COLUMN_PARENT_TOKEN + " STRING(MAX) NOT NULL,"
+ COLUMN_START_TIMESTAMP + " TIMESTAMP NOT NULL,"
+ COLUMN_INCLUSIVE_START + " BOOL NOT NULL, "
+ COLUMN_END_TIMESTAMP + " TIMESTAMP,"
+ COLUMN_INCLUSIVE_END + " BOOL,"
+ COLUMN_HEARTBEAT_SECONDS + " INT64 NOT NULL,"
+ COLUMN_STATE + " STRING(MAX) NOT NULL,"
+ COLUMN_CREATED_AT + " TIMESTAMP NOT NULL OPTIONS (allow_commit_timestamp=true),"
+ COLUMN_UPDATED_AT + " TIMESTAMP NOT NULL OPTIONS (allow_commit_timestamp=true)"
+ ") PRIMARY KEY (PartitionToken);";
+ COLUMN_PARTITION_TOKEN
+ " STRING(MAX) NOT NULL,"
+ COLUMN_PARENT_TOKEN
+ " ARRAY<STRING(MAX)> NOT NULL,"
+ COLUMN_START_TIMESTAMP
+ " TIMESTAMP NOT NULL,"
+ COLUMN_INCLUSIVE_START
+ " BOOL NOT NULL, "
+ COLUMN_END_TIMESTAMP
+ " TIMESTAMP,"
+ COLUMN_INCLUSIVE_END
+ " BOOL,"
+ COLUMN_HEARTBEAT_SECONDS
+ " INT64 NOT NULL,"
+ COLUMN_STATE
+ " STRING(MAX) NOT NULL,"
+ COLUMN_CREATED_AT
+ " TIMESTAMP NOT NULL OPTIONS (allow_commit_timestamp=true),"
+ COLUMN_UPDATED_AT
+ " TIMESTAMP NOT NULL OPTIONS (allow_commit_timestamp=true)"
+ ") PRIMARY KEY (PartitionToken)";
OperationFuture<Void, UpdateDatabaseDdlMetadata> op =
databaseAdminClient.updateDatabaseDdl(
id.getInstanceId().getInstance(),
Expand All @@ -94,16 +106,21 @@ private void createMetadataTable(DatabaseAdminClient databaseAdminClient, Databa
}
}

private void createFakeParentPartition(PartitionMetadataDao partitionMetadataDao,
DatabaseId id, Timestamp inclusiveStartAt, @Nullable Timestamp exclusiveEndAt) {
PartitionMetadata parentPartition = PartitionMetadata.newBuilder()
.setPartitionToken(DEFAULT_PARENT_PARTITION_TOKEN)
.setParentTokens(DEFAULT_PARENT_TOKENS)
.setStartTimestamp(inclusiveStartAt)
.setEndTimestamp(exclusiveEndAt)
.setHeartbeatSeconds(DEFAULT_HEARTBEAT_SECONDS)
.setState(State.CREATED)
.build();
partitionMetadataDao.insert(id.getDatabase(), parentPartition);
private static void createFakeParentPartition(
PartitionMetadataDao partitionMetadataDao,
Timestamp inclusiveStartAt,
@Nullable Timestamp exclusiveEndAt) {
PartitionMetadata parentPartition =
PartitionMetadata.newBuilder()
.setPartitionToken(DEFAULT_PARENT_PARTITION_TOKEN)
.setParentTokens(DEFAULT_PARENT_TOKENS)
.setStartTimestamp(inclusiveStartAt)
.setEndTimestamp(exclusiveEndAt)
.setHeartbeatSeconds(DEFAULT_HEARTBEAT_SECONDS)
.setState(State.CREATED)
.setCreatedAt(Value.COMMIT_TIMESTAMP)
.setUpdatedAt(Value.COMMIT_TIMESTAMP)
.build();
partitionMetadataDao.insert(parentPartition);
}
}
Original file line number Diff line number Diff line change
@@ -1,19 +1,20 @@
/*
* Copyright 2021 Google LLC
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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
*
* 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
* 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 org.apache.beam.sdk.io.gcp.spanner.cdc;

import com.google.cloud.Timestamp;
Expand All @@ -32,13 +33,13 @@
public class TimestampEncoding extends CustomEncoding<Timestamp> {

{
this.schema = SchemaBuilder
.builder()
.record("timestamp")
.fields()
.requiredLong("seconds")
.requiredInt("nanos")
.endRecord();
this.schema =
SchemaBuilder.builder()
.record("timestamp")
.fields()
.requiredLong("seconds")
.requiredInt("nanos")
.endRecord();
this.schema.addProp("CustomEncoding", TimestampEncoding.class.getSimpleName());
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,10 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.beam.sdk.io.gcp.spanner.cdc.dao;

import com.google.cloud.Timestamp;
import com.google.cloud.spanner.DatabaseClient;
import com.google.cloud.spanner.Mutation;
import com.google.common.collect.ImmutableList;
import org.apache.beam.sdk.io.gcp.spanner.cdc.model.PartitionMetadata;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;

public class PartitionMetadataDao {

Expand All @@ -21,34 +38,41 @@ public class PartitionMetadataDao {
public static final String COLUMN_UPDATED_AT = "UpdatedAt";

private final DatabaseClient databaseClient;
private final String tableName;

public PartitionMetadataDao (DatabaseClient databaseClient) {
public PartitionMetadataDao(DatabaseClient databaseClient, String tableName) {
this.databaseClient = databaseClient;
this.tableName = tableName;
}

public Timestamp insert(String table, PartitionMetadata partitionMetadata) {
Mutation mutation = Mutation.newInsertBuilder(table)
.set(COLUMN_PARTITION_TOKEN)
.to(partitionMetadata.getPartitionToken())
.set(COLUMN_PARENT_TOKEN)
.toStringArray(partitionMetadata.getParentTokens())
.set(COLUMN_START_TIMESTAMP)
.to(partitionMetadata.getStartTimestamp())
.set(COLUMN_INCLUSIVE_START)
.to(partitionMetadata.isInclusiveStart())
.set(COLUMN_END_TIMESTAMP)
.to(partitionMetadata.getEndTimestamp())
.set(COLUMN_INCLUSIVE_END)
.to(partitionMetadata.isInclusiveEnd())
.set(COLUMN_HEARTBEAT_SECONDS)
.to(partitionMetadata.getHeartbeatSeconds())
.set(COLUMN_STATE)
.to(partitionMetadata.getState().toString())
.set(COLUMN_CREATED_AT)
.to(partitionMetadata.getCreatedAt())
.set(COLUMN_UPDATED_AT)
.to(partitionMetadata.getUpdatedAt())
.build();
public String getTableName() {
return tableName;
}

public Timestamp insert(PartitionMetadata partitionMetadata) {
Mutation mutation =
Mutation.newInsertBuilder(this.tableName)
.set(COLUMN_PARTITION_TOKEN)
.to(partitionMetadata.getPartitionToken())
.set(COLUMN_PARENT_TOKEN)
.toStringArray(partitionMetadata.getParentTokens())
.set(COLUMN_START_TIMESTAMP)
.to(partitionMetadata.getStartTimestamp())
.set(COLUMN_INCLUSIVE_START)
.to(partitionMetadata.isInclusiveStart())
.set(COLUMN_END_TIMESTAMP)
.to(partitionMetadata.getEndTimestamp())
.set(COLUMN_INCLUSIVE_END)
.to(partitionMetadata.isInclusiveEnd())
.set(COLUMN_HEARTBEAT_SECONDS)
.to(partitionMetadata.getHeartbeatSeconds())
.set(COLUMN_STATE)
.to(partitionMetadata.getState().toString())
.set(COLUMN_CREATED_AT)
.to(partitionMetadata.getCreatedAt())
.set(COLUMN_UPDATED_AT)
.to(partitionMetadata.getUpdatedAt())
.build();
return databaseClient.write(ImmutableList.of(mutation));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/

/** User model for the Spanner change stream API. */
package org.apache.beam.sdk.io.gcp.spanner.cdc.dao;
Loading

0 comments on commit 0c9d0e2

Please sign in to comment.