Skip to content
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

feat: add the DeserializingSplitReader #11

Merged
merged 4 commits into from
Jul 16, 2021
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
4 changes: 4 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,10 @@
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</dependency>
<dependency>
<groupId>com.google.cloud</groupId>
<artifactId>google-cloud-core</artifactId>
</dependency>
<dependency>
<groupId>com.google.cloud</groupId>
<artifactId>google-cloud-pubsublite</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* Copyright 2021 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.pubsublite.flink;

import com.google.cloud.Timestamp;
import com.google.cloud.pubsublite.SequencedMessage;
import java.io.Serializable;

public interface MessageTimestampExtractor extends Serializable {
static MessageTimestampExtractor publishTimeExtractor() {
return (MessageTimestampExtractor) m -> Timestamp.fromProto(m.publishTime()).toDate().getTime();
}

static MessageTimestampExtractor eventTimeExtractor() {
return (MessageTimestampExtractor)
m -> {
if (m.message().eventTime().isPresent()) {
return Timestamp.fromProto(m.message().eventTime().get()).toDate().getTime();
}
return Timestamp.fromProto(m.publishTime()).toDate().getTime();
};
}

long timestamp(SequencedMessage m) throws Exception;
palmere-google marked this conversation as resolved.
Show resolved Hide resolved
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Copyright 2021 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.pubsublite.flink;

import com.google.cloud.pubsublite.SequencedMessage;
import java.io.Serializable;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;

public interface PubsubLiteDeserializationSchema<T> extends Serializable {

static <T> PubsubLiteDeserializationSchema<T> dataOnly(DeserializationSchema<T> schema) {
return new PubsubLiteDeserializationSchema<T>() {
@Override
public void open(DeserializationSchema.InitializationContext context) throws Exception {
schema.open(context);
}

@Override
public T deserialize(SequencedMessage message) throws Exception {
return schema.deserialize(message.message().data().toByteArray());
}

@Override
public TypeInformation<T> getProducedType() {
return schema.getProducedType();
}
};
}

void open(DeserializationSchema.InitializationContext context) throws Exception;

/**
* Deserialize a pub/sub lite message
*
* @param message The pub/sub lite message
* @return The deserialized message as an object (null if the message cannot be deserialized).
*/
T deserialize(SequencedMessage message) throws Exception;
Copy link
Contributor

Choose a reason for hiding this comment

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

Mark as @Nullable

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done


TypeInformation<T> getProducedType();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
/*
* Copyright 2021 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.pubsublite.flink.reader;

import com.google.cloud.pubsublite.SequencedMessage;
import com.google.cloud.pubsublite.flink.MessageTimestampExtractor;
import com.google.cloud.pubsublite.flink.PubsubLiteDeserializationSchema;
import com.google.cloud.pubsublite.flink.split.SubscriptionPartitionSplit;
import java.io.IOException;
import java.util.Optional;
import org.apache.flink.connector.base.source.reader.RecordsBySplits;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;

public class DeserializingSplitReader<T>
implements SplitReader<Record<T>, SubscriptionPartitionSplit> {
final SplitReader<SequencedMessage, SubscriptionPartitionSplit> reader;
palmere-google marked this conversation as resolved.
Show resolved Hide resolved
final PubsubLiteDeserializationSchema<T> schema;
final MessageTimestampExtractor selector;

public DeserializingSplitReader(
SplitReader<SequencedMessage, SubscriptionPartitionSplit> reader,
PubsubLiteDeserializationSchema<T> schema,
MessageTimestampExtractor selector) {
palmere-google marked this conversation as resolved.
Show resolved Hide resolved
this.reader = reader;
this.schema = schema;
this.selector = selector;
}

@Override
public RecordsBySplits<Record<T>> fetch() throws IOException {
RecordsBySplits.Builder<Record<T>> builder = new RecordsBySplits.Builder<>();
RecordsWithSplitIds<SequencedMessage> fetch = reader.fetch();
palmere-google marked this conversation as resolved.
Show resolved Hide resolved
for (String split = fetch.nextSplit(); split != null; split = fetch.nextSplit()) {
for (SequencedMessage m = fetch.nextRecordFromSplit();
m != null;
m = fetch.nextRecordFromSplit()) {
try {
T value = schema.deserialize(m);
long timestamp = selector.timestamp(m);
palmere-google marked this conversation as resolved.
Show resolved Hide resolved
builder.add(split, Record.create(Optional.ofNullable(value), m.offset(), timestamp));
} catch (Exception e) {
throw new IOException(e);
}
}
}
builder.addFinishedSplits(fetch.finishedSplits());
return builder.build();
}

@Override
public void handleSplitsChanges(SplitsChange<SubscriptionPartitionSplit> splitsChange) {
reader.handleSplitsChanges(splitsChange);
}

@Override
public void wakeUp() {
reader.wakeUp();
}

@Override
public void close() throws Exception {
reader.close();
}
}
33 changes: 33 additions & 0 deletions src/main/java/com/google/cloud/pubsublite/flink/reader/Record.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Copyright 2021 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.pubsublite.flink.reader;

import com.google.auto.value.AutoValue;
import com.google.cloud.pubsublite.Offset;
import java.util.Optional;

@AutoValue
dpcollins-google marked this conversation as resolved.
Show resolved Hide resolved
public abstract class Record<T> {
public abstract Optional<T> value();

public abstract Offset offset();

public abstract long timestamp();
Copy link
Contributor

Choose a reason for hiding this comment

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

use Instant until you absolutely need to convert to millis

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done


public static <T> Record<T> create(Optional<T> value, Offset offset, long timestamp) {
return new AutoValue_Record<>(value, offset, timestamp);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* Copyright 2021 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.pubsublite.flink.reader;

import static com.google.cloud.pubsublite.flink.TestUtilities.messageFromOffset;
import static com.google.cloud.pubsublite.flink.TestUtilities.recordWithSplitsToMap;
import static com.google.cloud.pubsublite.internal.testing.UnitTestExamples.examplePartition;
import static com.google.cloud.pubsublite.internal.testing.UnitTestExamples.exampleSubscriptionPath;
import static com.google.common.truth.Truth.assertThat;
import static org.junit.Assert.assertThrows;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

import com.google.cloud.pubsublite.Offset;
import com.google.cloud.pubsublite.SequencedMessage;
import com.google.cloud.pubsublite.flink.MessageTimestampExtractor;
import com.google.cloud.pubsublite.flink.PubsubLiteDeserializationSchema;
import com.google.cloud.pubsublite.flink.split.SubscriptionPartitionSplit;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Multimap;
import java.util.Optional;
import org.apache.flink.connector.base.source.reader.RecordsBySplits;
import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnitRunner;

@RunWith(MockitoJUnitRunner.class)
public class DeserializingSplitReaderTest {
static final SubscriptionPartitionSplit split =
SubscriptionPartitionSplit.create(
exampleSubscriptionPath(), examplePartition(), Offset.of(0));

@Mock PubsubLiteDeserializationSchema<String> mockDeserializationSchema;
@Mock MessageTimestampExtractor mockTimestampExtractor;
@Mock SplitReader<SequencedMessage, SubscriptionPartitionSplit> mockSplitReader;
DeserializingSplitReader<String> splitReader;

@Before
public void setUp() {
splitReader =
new DeserializingSplitReader<>(
mockSplitReader, mockDeserializationSchema, mockTimestampExtractor);
}

@Test
public void testDelegatingMethods() throws Exception {
SplitsAddition<SubscriptionPartitionSplit> addition =
new SplitsAddition<>(ImmutableList.of(split));
splitReader.handleSplitsChanges(addition);
verify(mockSplitReader).handleSplitsChanges(addition);

splitReader.close();
verify(mockSplitReader).close();

splitReader.wakeUp();
verify(mockSplitReader).wakeUp();
}

@Test
public void testDeserialization() throws Exception {
SequencedMessage message1 = messageFromOffset(Offset.of(10));
SequencedMessage message2 = messageFromOffset(Offset.of(20));

when(mockDeserializationSchema.deserialize(message1)).thenReturn("one");
when(mockDeserializationSchema.deserialize(message2)).thenReturn("two");
when(mockTimestampExtractor.timestamp(message1)).thenReturn(1L);
when(mockTimestampExtractor.timestamp(message2)).thenReturn(2L);

RecordsBySplits.Builder<SequencedMessage> records = new RecordsBySplits.Builder<>();
records.add("1", message1);
records.add("2", message2);
records.addFinishedSplit("finished");
when(mockSplitReader.fetch()).thenReturn(records.build());

RecordsBySplits<Record<String>> deserialized = splitReader.fetch();
assertThat(deserialized.finishedSplits()).containsExactly("finished");
Multimap<String, Record<String>> messages = recordWithSplitsToMap(deserialized);
assertThat(messages.get("1"))
.containsExactly(Record.create(Optional.of("one"), Offset.of(10), 1));
assertThat(messages.get("2"))
.containsExactly(Record.create(Optional.of("two"), Offset.of(20), 2));
}

@Test
public void testDeserializationReturnsNull() throws Exception {
SequencedMessage message1 = messageFromOffset(Offset.of(10));

when(mockDeserializationSchema.deserialize(message1)).thenReturn(null);
when(mockTimestampExtractor.timestamp(message1)).thenReturn(1L);

RecordsBySplits.Builder<SequencedMessage> records = new RecordsBySplits.Builder<>();
records.add("1", message1);
when(mockSplitReader.fetch()).thenReturn(records.build());

RecordsBySplits<Record<String>> deserialized = splitReader.fetch();
assertThat(deserialized.finishedSplits()).isEmpty();
Multimap<String, Record<String>> messages = recordWithSplitsToMap(deserialized);
assertThat(messages.get("1"))
.containsExactly(Record.create(Optional.empty(), Offset.of(10), 1));
}

@Test
public void testDeserializationFailure() throws Exception {
SequencedMessage message1 = messageFromOffset(Offset.of(10));

when(mockDeserializationSchema.deserialize(message1)).thenThrow(new Exception(""));

RecordsBySplits.Builder<SequencedMessage> records = new RecordsBySplits.Builder<>();
records.add("1", message1);
when(mockSplitReader.fetch()).thenReturn(records.build());

assertThrows(Exception.class, () -> splitReader.fetch());
}

@Test
public void testTimestampFailure() throws Exception {
SequencedMessage message1 = messageFromOffset(Offset.of(10));

when(mockDeserializationSchema.deserialize(message1)).thenReturn("one");
when(mockTimestampExtractor.timestamp(message1)).thenThrow(new Exception(""));

RecordsBySplits.Builder<SequencedMessage> records = new RecordsBySplits.Builder<>();
records.add("1", message1);
when(mockSplitReader.fetch()).thenReturn(records.build());

assertThrows(Exception.class, () -> splitReader.fetch());
}
}