Skip to content

Commit

Permalink
feat: Update CloudPubsubChecks to CloudPubsubTransforms (#730)
Browse files Browse the repository at this point in the history
* feat: Update CloudPubsubChecks to CloudPubsubTransforms

Also fix concerns from apache/beam#14917 surrounding direct use of MapElements.

Also add dependencyManagement section to beam io pom to fix upper bound deps issues.

* fix: Add pubsub proto dep
  • Loading branch information
dpcollins-google committed Jul 29, 2021
1 parent 759a7f1 commit 2e10924
Show file tree
Hide file tree
Showing 7 changed files with 191 additions and 53 deletions.
4 changes: 2 additions & 2 deletions pom.xml
Expand Up @@ -26,8 +26,8 @@
</dependency>
<dependency>
<groupId>com.google.cloud</groupId>
<artifactId>google-cloud-pubsub-bom</artifactId>
<version>1.113.5</version>
<artifactId>google-cloud-bom</artifactId>
<version>0.156.0</version>
<type>pom</type>
<scope>import</scope>
</dependency>
Expand Down
9 changes: 9 additions & 0 deletions pubsublite-beam-io/clirr-ignored-differences.xml
@@ -0,0 +1,9 @@
<?xml version="1.0" encoding="UTF-8"?>
<!-- see http://www.mojohaus.org/clirr-maven-plugin/examples/ignored-differences.html -->
<differences>
<!-- TODO: Remove on next release -->
<difference>
<differenceType>8001</differenceType>
<className>com/google/cloud/pubsublite/beam/CloudPubsubChecks</className>
</difference>
</differences>
39 changes: 39 additions & 0 deletions pubsublite-beam-io/pom.xml
Expand Up @@ -16,6 +16,36 @@
<name>Pub/Sub Lite IO</name>
<url>https://github.com/googleapis/java-pubsublite</url>
<description>Beam IO for Google Cloud Pub/Sub Lite</description>
<dependencyManagement>
<!-- Settings here exist to fix RequireUpperBoundDeps errors. -->
<dependencies>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>4.5.13</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
<version>4.4.14</version>
</dependency>
<dependency>
<groupId>com.google.apis</groupId>
<artifactId>google-api-services-bigquery</artifactId>
<version>v2-rev20210529-1.31.0</version>
</dependency>
<dependency>
<groupId>com.google.oauth-client</groupId>
<artifactId>google-oauth-client</artifactId>
<version>1.31.5</version>
</dependency>
<dependency>
<groupId>com.google.oauth-client</groupId>
<artifactId>google-oauth-client-java6</artifactId>
<version>1.31.5</version>
</dependency>
</dependencies>
</dependencyManagement>
<properties>
<beam.version>2.30.0</beam.version>
</properties>
Expand All @@ -30,6 +60,10 @@
<artifactId>proto-google-cloud-pubsublite-v1</artifactId>
<version>0.17.0</version><!-- {x-version-update:google-cloud-pubsublite:current} -->
</dependency>
<dependency>
<groupId>com.google.api.grpc</groupId>
<artifactId>proto-google-cloud-pubsub-v1</artifactId>
</dependency>
<dependency>
<groupId>com.google.auto.service</groupId>
<artifactId>auto-service-annotations</artifactId>
Expand Down Expand Up @@ -90,6 +124,11 @@
<artifactId>beam-sdks-java-core</artifactId>
<version>${beam.version}</version>
</dependency>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
<version>${beam.version}</version>
</dependency>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-extensions-protobuf</artifactId>
Expand Down

This file was deleted.

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

import com.google.protobuf.ByteString;
import java.util.Map;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* A class of utilities for transforming between Cloud Pub/Sub messages and the beam wrapper for
* them.
*/
class CloudPubsubMessages {
private static final Logger LOG = LoggerFactory.getLogger(CloudPubsubMessages.class);

private CloudPubsubMessages() {}

static com.google.pubsub.v1.PubsubMessage toProto(PubsubMessage input) {
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
Map<String, String> attributes = input.getAttributeMap();
if (attributes != null) {
message.putAllAttributes(attributes);
}
String messageId = input.getMessageId();
if (messageId != null) {
message.setMessageId(messageId);
}
return message.build();
}

static PubsubMessage fromProto(com.google.pubsub.v1.PubsubMessage proto) {
if (!proto.getOrderingKey().isEmpty()) {
LOG.warn(
"Dropping ordering key for message id `{}` with key `{}`.",
proto.getMessageId(),
proto.getOrderingKey());
}
return new PubsubMessage(
proto.getData().toByteArray(), proto.getAttributesMap(), proto.getMessageId());
}
}
@@ -0,0 +1,80 @@
/*
* Copyright 2020 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.beam;

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.proto.PubSubMessage;
import com.google.cloud.pubsublite.proto.SequencedMessage;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
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 ->
CloudPubsubMessages.fromProto(
toCpsSubscribeTransformer()
.transform(
com.google.cloud.pubsublite.SequencedMessage.fromProto(
message)))));
}
};
}
}
Expand Up @@ -105,7 +105,7 @@ public static PTransform<PCollection<PubSubMessage>, PCollection<PubSubMessage>>
* }</pre>
*/
public static PTransform<PCollection<PubSubMessage>, PDone> write(PublisherOptions options) {
return new PTransform<PCollection<PubSubMessage>, PDone>("PubsubLiteIO") {
return new PTransform<PCollection<PubSubMessage>, PDone>() {
@Override
public PDone expand(PCollection<PubSubMessage> input) {
PubsubLiteSink sink = new PubsubLiteSink(options);
Expand Down

0 comments on commit 2e10924

Please sign in to comment.