Skip to content
Open
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
Expand Up @@ -1977,11 +1977,6 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta
+ changeStreamDatabaseId
+ " has dialect "
+ changeStreamDatabaseDialect);
LOG.info(
"The Spanner database "
+ fullPartitionMetadataDatabaseId
+ " has dialect "
+ metadataDatabaseDialect);
PartitionMetadataTableNames partitionMetadataTableNames =
Optional.ofNullable(getMetadataTable())
.map(
Expand All @@ -2005,6 +2000,7 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta
final boolean isMutableChangeStream =
isMutableChangeStream(
spannerAccessor.getDatabaseClient(), changeStreamDatabaseDialect, changeStreamName);
LOG.info("The change stream " + changeStreamName + " is mutable: " + isMutableChangeStream);
final DaoFactory daoFactory =
new DaoFactory(
changeStreamSpannerConfig,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.cloud.Timestamp;
import com.google.cloud.spanner.ResultSet;
import com.google.cloud.spanner.Struct;
import com.google.protobuf.InvalidProtocolBufferException;
import org.joda.time.Duration;

/**
Expand Down Expand Up @@ -128,6 +129,33 @@ public boolean isProtoChangeRecord() {
&& resultSet.getColumnType(0).getCode() == com.google.cloud.spanner.Type.Code.PROTO;
}

/**
* Returns the change stream record at the current pointer by parsing the bytes column. It also
* updates the timestamp at which the record was read.
*
* <p>Should only be used for PostgreSQL databases when the change stream record is delivered as
* proto bytes.
*
* @return a change stream record as a proto or null
*/
public com.google.spanner.v1.ChangeStreamRecord getProtoChangeStreamRecordFromBytes() {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add accurate comments for this function, following similar convention for other functions? Thanks.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added comments. Please take a look.

recordReadAt = Timestamp.now();
try {
// Use getBytes(0) for the BYTES column returned by read_proto_bytes_ TVF
return com.google.spanner.v1.ChangeStreamRecord.parseFrom(
resultSet.getBytes(0).toByteArray());
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException("Failed to parse the proto bytes to ChangeStreamRecord proto", e);
}
}

/** Returns true if the result set at the current pointer contain only one bytes change record. */
public boolean isProtoBytesChangeRecord() {
return resultSet.getColumnCount() == 1
&& !resultSet.isNull(0)
&& resultSet.getColumnType(0).getCode() == com.google.cloud.spanner.Type.Code.BYTES;
}

/**
* Returns the record at the current pointer as {@link JsonB}. It also updates the timestamp at
* which the record was read.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,18 +218,28 @@ public class ChangeStreamRecordMapper {
* @param resultSet the change stream result set
* @param resultSetMetadata the metadata generated when reading the change stream row
* @return a {@link List} of {@link ChangeStreamRecord} subclasses
* @throws InvalidProtocolBufferException
*/
public List<ChangeStreamRecord> toChangeStreamRecords(
PartitionMetadata partition,
ChangeStreamResultSet resultSet,
ChangeStreamResultSetMetadata resultSetMetadata) {
if (this.isPostgres()) {
// In PostgresQL, change stream records are returned as JsonB.
// For `MUTABLE_KEY_RANGE` option, change stream records are returned as protos.
if (resultSet.isProtoBytesChangeRecord()) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we have the getBytes(0) reflected here to match up getPgJsonb(0) so that it is very clear
that we should only expect one column from the result set.

return Arrays.asList(
toChangeStreamRecord(
partition, resultSet.getProtoChangeStreamRecordFromBytes(), resultSetMetadata));
}

// For `IMMUTABLE_KEY_RANGE` option, change stream records are returned as
// JsonB.
return Collections.singletonList(
toChangeStreamRecordJson(partition, resultSet.getPgJsonb(0), resultSetMetadata));
}

// In GoogleSQL, for `MUTABLE_KEY_RANGE` option, change stream records are returned as Protos.
// In GoogleSQL, for `MUTABLE_KEY_RANGE` option, change stream records are
// returned as Protos.
if (resultSet.isProtoChangeRecord()) {
return Arrays.asList(
toChangeStreamRecord(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.changestreams.dao;

import static org.apache.beam.sdk.io.gcp.spanner.changestreams.util.TestProtoMapper.recordToProto;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

import com.google.cloud.ByteArray;
import com.google.cloud.Timestamp;
import com.google.cloud.spanner.ResultSet;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord;
import org.junit.Test;

public class ChangeStreamResultSetTest {

@Test
public void testGetProtoChangeStreamRecordFromBytes() throws Exception {
// 1. Create an expected ChangeStreamRecord proto
Timestamp now = Timestamp.now();
final HeartbeatRecord heartbeatRecord =
new HeartbeatRecord(Timestamp.ofTimeSecondsAndNanos(10L, 20), null);
com.google.spanner.v1.ChangeStreamRecord expectedRecord = recordToProto(heartbeatRecord);
assertNotNull(expectedRecord);

// 2. Convert it to bytes (simulating how Spanner PostgreSQL returns it)
byte[] protoBytes = expectedRecord.toByteArray();

// 3. Mock the underlying Spanner ResultSet
ResultSet mockResultSet = mock(ResultSet.class);
// Simulate column 0 containing the BYTES representation of the proto
when(mockResultSet.getBytes(0)).thenReturn(ByteArray.copyFrom(protoBytes));

// 4. Initialize ChangeStreamResultSet with the mock
ChangeStreamResultSet changeStreamResultSet = new ChangeStreamResultSet(mockResultSet);

// 5. Call the new method and assert it parses correctly
// (Note: This assumes you have added getProtoChangeStreamRecordFromBytes to the class)
com.google.spanner.v1.ChangeStreamRecord actualRecord =
changeStreamResultSet.getProtoChangeStreamRecordFromBytes();

assertEquals(expectedRecord, actualRecord);
}
}
Loading
Loading