Skip to content

Commit

Permalink
Port changes from Pub/Sub Lite to beam (#15418)
Browse files Browse the repository at this point in the history
* Port all changes from the Pub/Sub Lite repo back to beam.

Beam will be the canonical source for this IO in the future per offline discussion.

Also add the other direction of helper to CloudPubsubTransforms and add an integration test.

* remove fixed TODO

* Fixes to ReadWriteIT to work around Create or DirectRunner bug.

* fix racy test
  • Loading branch information
dpcollins-google committed Sep 10, 2021
1 parent efb4b60 commit 8a646aa
Show file tree
Hide file tree
Showing 26 changed files with 939 additions and 298 deletions.
Expand Up @@ -446,7 +446,7 @@ class BeamModulePlugin implements Plugin<Project> {
def errorprone_version = "2.3.4"
def google_clients_version = "1.32.1"
def google_cloud_bigdataoss_version = "2.2.2"
def google_cloud_pubsublite_version = "0.13.2"
def google_cloud_pubsublite_version = "1.0.4"
def google_code_gson_version = "2.8.6"
def google_oauth_clients_version = "1.31.0"
// Try to keep grpc_version consistent with gRPC version in google_cloud_platform_libraries_bom
Expand Down
Expand Up @@ -24,25 +24,36 @@
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;

/** Common util functions for converting between PubsubMessage proto and {@link PubsubMessage}. */
public class PubsubMessages {
public final class PubsubMessages {
private PubsubMessages() {}

public static com.google.pubsub.v1.PubsubMessage toProto(PubsubMessage input) {
Map<String, String> attributes = input.getAttributeMap();
com.google.pubsub.v1.PubsubMessage.Builder message =
com.google.pubsub.v1.PubsubMessage.newBuilder()
.setData(ByteString.copyFrom(input.getPayload()));
// TODO(BEAM-8085) this should not be null
if (attributes != null) {
message.putAllAttributes(attributes);
}
String messageId = input.getMessageId();
if (messageId != null) {
message.setMessageId(messageId);
}
return message.build();
}

public static PubsubMessage fromProto(com.google.pubsub.v1.PubsubMessage input) {
return new PubsubMessage(
input.getData().toByteArray(), input.getAttributesMap(), input.getMessageId());
}

// Convert the PubsubMessage to a PubsubMessage proto, then return its serialized representation.
public static class ParsePayloadAsPubsubMessageProto
implements SerializableFunction<PubsubMessage, byte[]> {
@Override
public byte[] apply(PubsubMessage input) {
Map<String, String> attributes = input.getAttributeMap();
com.google.pubsub.v1.PubsubMessage.Builder message =
com.google.pubsub.v1.PubsubMessage.newBuilder()
.setData(ByteString.copyFrom(input.getPayload()));
// TODO(BEAM-8085) this should not be null
if (attributes != null) {
message.putAllAttributes(attributes);
}
String messageId = input.getMessageId();
if (messageId != null) {
message.setMessageId(messageId);
}
return message.build().toByteArray();
return toProto(input).toByteArray();
}
}

Expand All @@ -54,8 +65,7 @@ public PubsubMessage apply(byte[] input) {
try {
com.google.pubsub.v1.PubsubMessage message =
com.google.pubsub.v1.PubsubMessage.parseFrom(input);
return new PubsubMessage(
message.getData().toByteArray(), message.getAttributesMap(), message.getMessageId());
return fromProto(message);
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException("Could not decode Pubsub message", e);
}
Expand Down

This file was deleted.

@@ -0,0 +1,104 @@
/*
* 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.pubsublite;

import static com.google.cloud.pubsublite.cloudpubsub.MessageTransforms.fromCpsPublishTransformer;
import static com.google.cloud.pubsublite.cloudpubsub.MessageTransforms.toCpsPublishTransformer;
import static com.google.cloud.pubsublite.cloudpubsub.MessageTransforms.toCpsSubscribeTransformer;

import com.google.cloud.pubsublite.Message;
import com.google.cloud.pubsublite.cloudpubsub.KeyExtractor;
import com.google.cloud.pubsublite.proto.PubSubMessage;
import com.google.cloud.pubsublite.proto.SequencedMessage;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessages;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TypeDescriptor;

/** A class providing transforms between Cloud Pub/Sub and Pub/Sub Lite message types. */
public final class CloudPubsubTransforms {
private CloudPubsubTransforms() {}
/**
* Ensure that all messages that pass through can be converted to Cloud Pub/Sub messages using the
* standard transformation methods in the client library.
*
* <p>Will fail the pipeline if a message has multiple attributes per key.
*/
public static PTransform<PCollection<PubSubMessage>, PCollection<PubSubMessage>>
ensureUsableAsCloudPubsub() {
return new PTransform<PCollection<PubSubMessage>, PCollection<PubSubMessage>>() {
@Override
public PCollection<PubSubMessage> expand(PCollection<PubSubMessage> input) {
return input.apply(
MapElements.into(TypeDescriptor.of(PubSubMessage.class))
.via(
message -> {
Object unused =
toCpsPublishTransformer().transform(Message.fromProto(message));
return message;
}));
}
};
}

/**
* Transform messages read from Pub/Sub Lite to their equivalent Cloud Pub/Sub Message that would
* have been read from PubsubIO.
*
* <p>Will fail the pipeline if a message has multiple attributes per map key.
*/
public static PTransform<PCollection<SequencedMessage>, PCollection<PubsubMessage>>
toCloudPubsubMessages() {
return new PTransform<PCollection<SequencedMessage>, PCollection<PubsubMessage>>() {
@Override
public PCollection<PubsubMessage> expand(PCollection<SequencedMessage> input) {
return input.apply(
MapElements.into(TypeDescriptor.of(PubsubMessage.class))
.via(
message ->
PubsubMessages.fromProto(
toCpsSubscribeTransformer()
.transform(
com.google.cloud.pubsublite.SequencedMessage.fromProto(
message)))));
}
};
}

/**
* Transform messages publishable using PubsubIO to their equivalent Pub/Sub Lite publishable
* message.
*/
public static PTransform<PCollection<PubsubMessage>, PCollection<PubSubMessage>>
fromCloudPubsubMessages() {
return new PTransform<PCollection<PubsubMessage>, PCollection<PubSubMessage>>() {
@Override
public PCollection<PubSubMessage> expand(PCollection<PubsubMessage> input) {
return input.apply(
MapElements.into(TypeDescriptor.of(PubSubMessage.class))
.via(
message ->
fromCpsPublishTransformer(KeyExtractor.DEFAULT)
.transform(PubsubMessages.toProto(message))
.toProto()));
}
};
}
}
@@ -0,0 +1,33 @@
/*
* 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.pubsublite;

import java.io.Serializable;

/**
* A ManagedBacklogReaderFactory produces TopicBacklogReaders and tears down any produced readers
* when it is itself closed.
*
* <p>close() should never be called on produced readers.
*/
public interface ManagedBacklogReaderFactory extends AutoCloseable, Serializable {
TopicBacklogReader newReader(SubscriptionPartition subscriptionPartition);

@Override
void close();
}
@@ -0,0 +1,68 @@
/*
* 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.pubsublite;

import com.google.api.gax.rpc.ApiException;
import com.google.cloud.pubsublite.Offset;
import com.google.cloud.pubsublite.proto.ComputeMessageStatsResponse;
import java.util.HashMap;
import java.util.Map;
import javax.annotation.concurrent.GuardedBy;
import org.apache.beam.sdk.transforms.SerializableFunction;

public class ManagedBacklogReaderFactoryImpl implements ManagedBacklogReaderFactory {
private final SerializableFunction<SubscriptionPartition, TopicBacklogReader> newReader;

@GuardedBy("this")
private final Map<SubscriptionPartition, TopicBacklogReader> readers = new HashMap<>();

ManagedBacklogReaderFactoryImpl(
SerializableFunction<SubscriptionPartition, TopicBacklogReader> newReader) {
this.newReader = newReader;
}

private static final class NonCloseableTopicBacklogReader implements TopicBacklogReader {
private final TopicBacklogReader underlying;

NonCloseableTopicBacklogReader(TopicBacklogReader underlying) {
this.underlying = underlying;
}

@Override
public ComputeMessageStatsResponse computeMessageStats(Offset offset) throws ApiException {
return underlying.computeMessageStats(offset);
}

@Override
public void close() {
throw new IllegalArgumentException(
"Cannot call close() on a reader returned from ManagedBacklogReaderFactory.");
}
}

@Override
public synchronized TopicBacklogReader newReader(SubscriptionPartition subscriptionPartition) {
return new NonCloseableTopicBacklogReader(
readers.computeIfAbsent(subscriptionPartition, newReader::apply));
}

@Override
public synchronized void close() {
readers.values().forEach(TopicBacklogReader::close);
}
}
@@ -0,0 +1,38 @@
/*
* 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.pubsublite;

import com.google.auto.value.AutoValue;
import org.apache.beam.sdk.coders.DefaultCoder;
import org.apache.beam.sdk.io.range.OffsetRange;

@AutoValue
@DefaultCoder(OffsetByteRangeCoder.class)
abstract class OffsetByteRange {
abstract OffsetRange getRange();

abstract long getByteCount();

static OffsetByteRange of(OffsetRange range, long byteCount) {
return new AutoValue_OffsetByteRange(range, byteCount);
}

static OffsetByteRange of(OffsetRange range) {
return of(range, 0);
}
}

0 comments on commit 8a646aa

Please sign in to comment.