Skip to content
Closed
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
@@ -0,0 +1,131 @@
/*
* 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 com.google.auto.service.AutoService;
import com.google.protobuf.ByteString;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.PubsubSubscription;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.PubsubTopic;
import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;

/** Exposes {@link PubsubIO.Read} as an external transform for cross-language usage. */
@Experimental
@AutoService(ExternalTransformRegistrar.class)
public final class ExternalRead implements ExternalTransformRegistrar {
public ExternalRead() {}

public static final String URN = "beam:external:java:pubsub:read:v1";

@Override
public Map<String, Class<? extends ExternalTransformBuilder>> knownBuilders() {
return ImmutableMap.of(URN, ReadBuilder.class);
}

/** Parameters class to expose the transform to an external SDK. */
public static class Configuration {
@Nullable private String topic;
@Nullable private String subscription;
@Nullable private String idAttribute;
@Nullable private String timestampAttribute;
private boolean needsAttributes;

public void setTopic(@Nullable String topic) {
this.topic = topic;
}

public void setSubscription(@Nullable String subscription) {
this.subscription = subscription;
}

public void setIdLabel(@Nullable String idAttribute) {
this.idAttribute = idAttribute;
}

public void setTimestampAttribute(@Nullable String timestampAttribute) {
this.timestampAttribute = timestampAttribute;
}

public void setWithAttributes(Boolean needsAttributes) {
// we must use Boolean instead of boolean because the external payload system
// inspects the native type of each coder urn, and BooleanCoder wants Boolean.
this.needsAttributes = needsAttributes;
}
}

public static class ReadBuilder
implements ExternalTransformBuilder<Configuration, PBegin, PCollection<byte[]>> {
public ReadBuilder() {}

@Override
public PTransform<PBegin, PCollection<byte[]>> buildExternal(Configuration config) {
PubsubIO.Read.Builder<byte[]> readBuilder;
if (config.needsAttributes) {
readBuilder = PubsubIO.Read.newBuilder(new ParsePayloadAsPubsubMessageProto());
readBuilder.setNeedsAttributes(true);
} else {
readBuilder = PubsubIO.Read.newBuilder(PubsubMessage::getPayload);
}
readBuilder.setCoder(ByteArrayCoder.of());
if (config.topic != null) {
StaticValueProvider<String> topic = StaticValueProvider.of(config.topic);
readBuilder.setTopicProvider(NestedValueProvider.of(topic, PubsubTopic::fromPath));
}
if (config.subscription != null) {
StaticValueProvider<String> subscription = StaticValueProvider.of(config.subscription);
readBuilder.setSubscriptionProvider(
NestedValueProvider.of(subscription, PubsubSubscription::fromPath));
}
if (config.idAttribute != null) {
readBuilder.setIdAttribute(config.idAttribute);
}
if (config.timestampAttribute != null) {
readBuilder.setTimestampAttribute(config.timestampAttribute);
}
return readBuilder.build();
}
}

// Convert the PubsubMessage to a PubsubMessage proto, then return its serialized representation.
private 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);
}
return message.build().toByteArray();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
/*
* 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 com.google.auto.service.AutoService;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.pubsub.v1.PubsubMessage;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.PubsubTopic;
import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;

/** Exposes {@link PubsubIO.Write} as an external transform for cross-language usage. */
@Experimental
@AutoService(ExternalTransformRegistrar.class)
public final class ExternalWrite implements ExternalTransformRegistrar {
public ExternalWrite() {}

public static final String URN = "beam:external:java:pubsub:write:v1";

@Override
public Map<String, Class<? extends ExternalTransformBuilder>> knownBuilders() {
return ImmutableMap.of(URN, WriteBuilder.class);
}

/** Parameters class to expose the transform to an external SDK. */
public static class Configuration {
private String topic;
@Nullable private String idAttribute;
@Nullable private String timestampAttribute;

public void setTopic(String topic) {
this.topic = topic;
}

public void setIdLabel(@Nullable String idAttribute) {
this.idAttribute = idAttribute;
}

public void setTimestampAttribute(@Nullable String timestampAttribute) {
this.timestampAttribute = timestampAttribute;
}
}

public static class WriteBuilder
implements ExternalTransformBuilder<Configuration, PCollection<byte[]>, PDone> {
public WriteBuilder() {}

@Override
public PTransform<PCollection<byte[]>, PDone> buildExternal(Configuration config) {
PubsubIO.Write.Builder<byte[]> writeBuilder = PubsubIO.Write.newBuilder(new FormatFn());
if (config.topic != null) {
StaticValueProvider<String> topic = StaticValueProvider.of(config.topic);
writeBuilder.setTopicProvider(NestedValueProvider.of(topic, PubsubTopic::fromPath));
}
if (config.idAttribute != null) {
writeBuilder.setIdAttribute(config.idAttribute);
}
if (config.timestampAttribute != null) {
writeBuilder.setTimestampAttribute(config.timestampAttribute);
}
return writeBuilder.build();
}
}

private static class FormatFn implements SerializableFunction<byte[], PubsubMessage> {
@Override
public PubsubMessage apply(byte[] input) {
try {
return PubsubMessage.parseFrom(input);
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException("Could not decode Pubsub message", e);
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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 com.google.auto.value.AutoValue;
import com.google.pubsub.v1.PubsubMessage;
import org.apache.beam.sdk.coders.DefaultCoder;

/** A message received from Pubsub. */
@AutoValue
@DefaultCoder(IncomingMessageCoder.class)
abstract class IncomingMessage {

/** Underlying Message. */
public abstract com.google.pubsub.v1.PubsubMessage message();

/**
* Timestamp for element (ms since epoch). Either Pubsub's processing time, or the custom
* timestamp associated with the message.
*/
public abstract long timestampMsSinceEpoch();

/** Timestamp (in system time) at which we requested the message (ms since epoch). */
public abstract long requestTimeMsSinceEpoch();

/** Id to pass back to Pubsub to acknowledge receipt of this message. */
public abstract String ackId();

/** Id to pass to the runner to distinguish this message from all others. */
public abstract String recordId();

public static IncomingMessage of(
PubsubMessage message,
long timestampMsSinceEpoch,
long requestTimeMsSinceEpoch,
String ackId,
String recordId) {
return new AutoValue_IncomingMessage(
message, timestampMsSinceEpoch, requestTimeMsSinceEpoch, ackId, recordId);
}
}
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.pubsub;

import com.google.pubsub.v1.PubsubMessage;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import org.apache.beam.sdk.coders.CoderProvider;
import org.apache.beam.sdk.coders.CoderProviders;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
import org.apache.beam.sdk.values.TypeDescriptor;

/** A coder for IncomingMessage. */
public class IncomingMessageCoder extends CustomCoder<IncomingMessage> {
public static CoderProvider getCoderProvider() {
return CoderProviders.forCoder(
TypeDescriptor.of(IncomingMessage.class), new IncomingMessageCoder());
}

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

@Override
public void encode(IncomingMessage value, OutputStream outStream) throws IOException {
ProtoCoder.of(PubsubMessage.class).encode(value.message(), outStream);
VarLongCoder.of().encode(value.timestampMsSinceEpoch(), outStream);
VarLongCoder.of().encode(value.requestTimeMsSinceEpoch(), outStream);
StringUtf8Coder.of().encode(value.ackId(), outStream);
StringUtf8Coder.of().encode(value.recordId(), outStream);
}

@Override
public IncomingMessage decode(InputStream inStream) throws IOException {
return IncomingMessage.of(
ProtoCoder.of(PubsubMessage.class).decode(inStream),
VarLongCoder.of().decode(inStream),
VarLongCoder.of().decode(inStream),
StringUtf8Coder.of().decode(inStream),
StringUtf8Coder.of().decode(inStream));
}
}
Loading