Skip to content
Merged
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 @@ -100,6 +100,7 @@
import org.apache.beam.sdk.io.WriteFilesResult;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageWithAttributesAndMessageIdCoder;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageWithAttributesCoder;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSource;
Expand Down Expand Up @@ -1161,11 +1162,12 @@ PubsubUnboundedSource getOverriddenTransform() {

@Override
public PCollection<PubsubMessage> expand(PBegin input) {
Coder coder =
transform.getNeedsMessageId()
? new PubsubMessageWithAttributesAndMessageIdCoder()
: new PubsubMessageWithAttributesCoder();
return PCollection.createPrimitiveOutputInternal(
input.getPipeline(),
WindowingStrategy.globalDefault(),
IsBounded.UNBOUNDED,
new PubsubMessageWithAttributesCoder());
input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, coder);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,9 @@ protected WindowedValue<T> decodeMessage(Windmill.Message message) throws IOExce
value =
parseFn.apply(
new PubsubMessage(
pubsubMessage.getData().toByteArray(), pubsubMessage.getAttributes()));
pubsubMessage.getData().toByteArray(),
pubsubMessage.getAttributesMap(),
pubsubMessage.getMessageId()));
} else {
value = coder.decode(data, Coder.Context.OUTER);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,11 @@ public class PubsubCoderProviderRegistrar implements CoderProviderRegistrar {
public List<CoderProvider> getCoderProviders() {
return ImmutableList.of(
CoderProviders.forCoder(
TypeDescriptor.of(PubsubMessage.class), PubsubMessageWithAttributesCoder.of()));
TypeDescriptor.of(PubsubMessage.class), PubsubMessageWithAttributesCoder.of()),
CoderProviders.forCoder(
TypeDescriptor.of(PubsubMessage.class), PubsubMessageWithMessageIdCoder.of()),
CoderProviders.forCoder(
TypeDescriptor.of(PubsubMessage.class),
PubsubMessageWithAttributesAndMessageIdCoder.of()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -441,6 +441,7 @@ public String toString() {
private static <T> Read<T> read() {
return new AutoValue_PubsubIO_Read.Builder<T>()
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.setPubsubClientFactory(FACTORY)
.build();
}
Expand All @@ -456,6 +457,23 @@ public static Read<PubsubMessage> readMessages() {
.setCoder(PubsubMessagePayloadOnlyCoder.of())
.setParseFn(new IdentityMessageFn())
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.build();
}

/**
* Returns A {@link PTransform} that continuously reads from a Google Cloud Pub/Sub stream. The
* messages will only contain a {@link PubsubMessage#getPayload() payload} with the {@link
* PubsubMessage#getMessageId() messageId} from PubSub, but no {@link
* PubsubMessage#getAttributeMap() attributes}.
*/
public static Read<PubsubMessage> readMessagesWithMessageId() {
return new AutoValue_PubsubIO_Read.Builder<PubsubMessage>()
.setPubsubClientFactory(FACTORY)
.setCoder(PubsubMessageWithMessageIdCoder.of())
.setParseFn(new IdentityMessageFn())
.setNeedsAttributes(false)
.setNeedsMessageId(true)
.build();
}

Expand All @@ -470,6 +488,23 @@ public static Read<PubsubMessage> readMessagesWithAttributes() {
.setCoder(PubsubMessageWithAttributesCoder.of())
.setParseFn(new IdentityMessageFn())
.setNeedsAttributes(true)
.setNeedsMessageId(false)
.build();
}

/**
* Returns A {@link PTransform} that continuously reads from a Google Cloud Pub/Sub stream. The
* messages will contain both a {@link PubsubMessage#getPayload() payload} and {@link
* PubsubMessage#getAttributeMap() attributes}, along with the {@link PubsubMessage#getMessageId()
* messageId} from PubSub.
*/
public static Read<PubsubMessage> readMessagesWithAttributesAndMessageId() {
return new AutoValue_PubsubIO_Read.Builder<PubsubMessage>()
.setPubsubClientFactory(FACTORY)
.setCoder(PubsubMessageWithAttributesAndMessageIdCoder.of())
.setParseFn(new IdentityMessageFn())
.setNeedsAttributes(true)
.setNeedsMessageId(true)
.build();
}

Expand All @@ -480,6 +515,7 @@ public static Read<PubsubMessage> readMessagesWithAttributes() {
public static Read<String> readStrings() {
return new AutoValue_PubsubIO_Read.Builder<String>()
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.setPubsubClientFactory(FACTORY)
.setCoder(StringUtf8Coder.of())
.setParseFn(new ParsePayloadAsUtf8())
Expand All @@ -497,6 +533,7 @@ public static <T extends Message> Read<T> readProtos(Class<T> messageClass) {
ProtoCoder<T> coder = ProtoCoder.of(messageClass);
return new AutoValue_PubsubIO_Read.Builder<T>()
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.setPubsubClientFactory(FACTORY)
.setCoder(coder)
.setParseFn(new ParsePayloadUsingCoder<>(coder))
Expand All @@ -514,6 +551,7 @@ public static <T> Read<T> readAvros(Class<T> clazz) {
AvroCoder<T> coder = AvroCoder.of(clazz);
return new AutoValue_PubsubIO_Read.Builder<T>()
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.setPubsubClientFactory(FACTORY)
.setCoder(coder)
.setParseFn(new ParsePayloadUsingCoder<>(coder))
Expand All @@ -533,6 +571,7 @@ public static Read<GenericRecord> readAvroGenericRecords(org.apache.avro.Schema
AvroCoder<GenericRecord> coder = AvroCoder.of(GenericRecord.class, avroSchema);
return new AutoValue_PubsubIO_Read.Builder<GenericRecord>()
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.setPubsubClientFactory(FACTORY)
.setBeamSchema(schema)
.setToRowFn(AvroUtils.getToRowFunction(GenericRecord.class, avroSchema))
Expand All @@ -558,6 +597,7 @@ public static <T> Read<T> readAvrosWithBeamSchema(Class<T> clazz) {
Schema schema = AvroUtils.getSchema(clazz, null);
return new AutoValue_PubsubIO_Read.Builder<T>()
.setNeedsAttributes(false)
.setNeedsMessageId(false)
.setPubsubClientFactory(FACTORY)
.setBeamSchema(schema)
.setToRowFn(AvroUtils.getToRowFunction(clazz, avroSchema))
Expand Down Expand Up @@ -645,6 +685,8 @@ public abstract static class Read<T> extends PTransform<PBegin, PCollection<T>>

abstract boolean getNeedsAttributes();

abstract boolean getNeedsMessageId();

abstract Builder<T> toBuilder();

@AutoValue.Builder
Expand All @@ -671,6 +713,8 @@ abstract static class Builder<T> {

abstract Builder<T> setNeedsAttributes(boolean needsAttributes);

abstract Builder<T> setNeedsMessageId(boolean needsMessageId);

abstract Builder<T> setClock(@Nullable Clock clock);

abstract Read<T> build();
Expand Down Expand Up @@ -832,7 +876,8 @@ public PCollection<T> expand(PBegin input) {
subscriptionPath,
getTimestampAttribute(),
getIdAttribute(),
getNeedsAttributes());
getNeedsAttributes(),
getNeedsMessageId());
PCollection<T> read = input.apply(source).apply(MapElements.via(getParseFn()));
return (getBeamSchema() != null)
? read.setSchema(getBeamSchema(), getToRowFn(), getFromRowFn())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,17 +23,25 @@
import javax.annotation.Nullable;

/**
* Class representing a Pub/Sub message. Each message contains a single message payload and a map of
* attached attributes.
* Class representing a Pub/Sub message. Each message contains a single message payload, a map of
* attached attributes, and a message id.
*/
public class PubsubMessage {

private byte[] message;
private Map<String, String> attributes;
private String messageId;

public PubsubMessage(byte[] payload, Map<String, String> attributes) {
this.message = payload;
this.attributes = attributes;
this.messageId = null;
}

public PubsubMessage(byte[] payload, Map<String, String> attributes, String messageId) {
this.message = payload;
this.attributes = attributes;
this.messageId = messageId;
}

/** Returns the main PubSub message. */
Expand All @@ -52,4 +60,10 @@ public String getAttribute(String attribute) {
public Map<String, String> getAttributeMap() {
return attributes;
}

/** Returns the messageId of the message populated by Cloud Pub/Sub. */
@Nullable
public String getMessageId() {
return messageId;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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.pubsub;

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Map;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.MapCoder;
import org.apache.beam.sdk.coders.NullableCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.values.TypeDescriptor;

/** A coder for PubsubMessage including attributes and the message id from the PubSub server. */
public class PubsubMessageWithAttributesAndMessageIdCoder extends CustomCoder<PubsubMessage> {
// A message's payload cannot be null
private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
// A message's attributes can be null.
private static final Coder<Map<String, String>> ATTRIBUTES_CODER =
NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
// A message's messageId cannot be null
private static final Coder<String> MESSAGE_ID_CODER = StringUtf8Coder.of();

public static Coder<PubsubMessage> of(TypeDescriptor<PubsubMessage> ignored) {
return of();
}

public static PubsubMessageWithAttributesAndMessageIdCoder of() {
return new PubsubMessageWithAttributesAndMessageIdCoder();
}

@Override
public void encode(PubsubMessage value, OutputStream outStream) throws IOException {
PAYLOAD_CODER.encode(value.getPayload(), outStream);
ATTRIBUTES_CODER.encode(value.getAttributeMap(), outStream);
MESSAGE_ID_CODER.encode(value.getMessageId(), outStream);
}

@Override
public PubsubMessage decode(InputStream inStream) throws IOException {
byte[] payload = PAYLOAD_CODER.decode(inStream);
Map<String, String> attributes = ATTRIBUTES_CODER.decode(inStream);
String messageId = MESSAGE_ID_CODER.decode(inStream);
return new PubsubMessage(payload, attributes, messageId);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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.pubsub;

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;

/**
* A coder for PubsubMessage treating the raw bytes being decoded as the message's payload, with the
* message id from the PubSub server.
*/
public class PubsubMessageWithMessageIdCoder extends CustomCoder<PubsubMessage> {
private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
// A message's messageId cannot be null
private static final Coder<String> MESSAGE_ID_CODER = StringUtf8Coder.of();

public static PubsubMessageWithMessageIdCoder of() {
return new PubsubMessageWithMessageIdCoder();
}

@Override
public void encode(PubsubMessage value, OutputStream outStream) throws IOException {
PAYLOAD_CODER.encode(value.getPayload(), outStream);
MESSAGE_ID_CODER.encode(value.getMessageId(), outStream);
}

@Override
public PubsubMessage decode(InputStream inStream) throws IOException {
byte[] payload = PAYLOAD_CODER.decode(inStream);
String messageId = MESSAGE_ID_CODER.decode(inStream);
return new PubsubMessage(payload, ImmutableMap.of(), messageId);
}
}
Loading