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

Add portable Mqtt source and sink transforms #32385

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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
2 changes: 2 additions & 0 deletions sdks/java/io/expansion-service/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@ dependencies {
permitUnusedDeclared project(":sdks:java:io:kafka") // BEAM-11761
implementation project(":sdks:java:io:kafka:upgrade")
permitUnusedDeclared project(":sdks:java:io:kafka:upgrade") // BEAM-11761
implementation project(":sdks:java:io:mqtt")
permitUnusedDeclared project(":sdks:java:io:mqtt") // BEAM-11761

// **** IcebergIO runtime dependencies ****
runtimeOnly library.java.hadoop_client
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,9 @@
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
Expand Down Expand Up @@ -120,13 +123,17 @@ public static Write write() {
private MqttIO() {}

/** A POJO describing a MQTT connection. */
@DefaultSchema(AutoValueSchema.class)
@AutoValue
public abstract static class ConnectionConfiguration implements Serializable {

@SchemaFieldDescription("The MQTT broker URI.")
abstract String getServerUri();

@SchemaFieldDescription("The MQTT topic pattern.")
abstract String getTopic();

@SchemaFieldDescription("The client ID prefix, which is used to construct a unique client ID.")
abstract @Nullable String getClientId();

abstract @Nullable String getUsername();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* 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.mqtt;

import static org.apache.beam.sdk.io.mqtt.MqttIO.ConnectionConfiguration;
import static org.apache.beam.sdk.io.mqtt.MqttReadSchemaTransformProvider.ReadConfiguration;

import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
import java.io.Serializable;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.joda.time.Duration;

@AutoService(SchemaTransformProvider.class)
public class MqttReadSchemaTransformProvider
extends TypedSchemaTransformProvider<ReadConfiguration> {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
public abstract static class ReadConfiguration implements Serializable {
public static Builder builder() {
return new AutoValue_MqttReadSchemaTransformProvider_ReadConfiguration.Builder();
}

@SchemaFieldDescription("Configuration options to set up the MQTT connection.")
public abstract ConnectionConfiguration getConnectionConfiguration();

@SchemaFieldDescription(
"The max number of records to receive. Setting this will result in a bounded PCollection.")
@Nullable
public abstract Long getMaxNumRecords();

@SchemaFieldDescription(
"The maximum time for this source to read messages. Setting this will result in a bounded PCollection.")
@Nullable
public abstract Long getMaxReadTimeSeconds();

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setConnectionConfiguration(
ConnectionConfiguration connectionConfiguration);

public abstract Builder setMaxNumRecords(Long maxNumRecords);

public abstract Builder setMaxReadTimeSeconds(Long maxReadTimeSeconds);

public abstract ReadConfiguration build();
}
}

@Override
public String identifier() {
return "beam:schematransform:org.apache.beam:mqtt_read:v1";
}

@Override
protected SchemaTransform from(ReadConfiguration configuration) {
return new MqttReadSchemaTransform(configuration);
}

private static class MqttReadSchemaTransform extends SchemaTransform {
private final ReadConfiguration config;

MqttReadSchemaTransform(ReadConfiguration configuration) {
this.config = configuration;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
Preconditions.checkState(
input.getAll().isEmpty(),
"Expected zero input PCollections for this source, but found: %",
input.getAll().keySet());

MqttIO.Read readTransform =
MqttIO.read().withConnectionConfiguration(config.getConnectionConfiguration());

Long maxRecords = config.getMaxNumRecords();
Long maxReadTime = config.getMaxReadTimeSeconds();
if (maxRecords != null) {
readTransform = readTransform.withMaxNumRecords(maxRecords);
}
if (maxReadTime != null) {
readTransform = readTransform.withMaxReadTime(Duration.standardSeconds(maxReadTime));
}

Schema outputSchema = Schema.builder().addByteArrayField("bytes").build();

PCollection<Row> outputRows =
input
.getPipeline()
.apply(readTransform)
.apply(
"Wrap in Beam Rows",
ParDo.of(
new DoFn<byte[], Row>() {
@ProcessElement
public void processElement(
@Element byte[] data, OutputReceiver<Row> outputReceiver) {
outputReceiver.output(
Row.withSchema(outputSchema).addValue(data).build());
}
}))
.setRowSchema(outputSchema);

return PCollectionRowTuple.of("output", outputRows);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* 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.mqtt;

import static org.apache.beam.sdk.io.mqtt.MqttIO.ConnectionConfiguration;
import static org.apache.beam.sdk.io.mqtt.MqttWriteSchemaTransformProvider.WriteConfiguration;

import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
import java.io.Serializable;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;

@AutoService(SchemaTransformProvider.class)
public class MqttWriteSchemaTransformProvider
extends TypedSchemaTransformProvider<WriteConfiguration> {
@DefaultSchema(AutoValueSchema.class)
@AutoValue
public abstract static class WriteConfiguration implements Serializable {
public static Builder builder() {
return new AutoValue_MqttWriteSchemaTransformProvider_WriteConfiguration.Builder();
}

@SchemaFieldDescription("Configuration options to set up the MQTT connection.")
public abstract ConnectionConfiguration getConnectionConfiguration();

@SchemaFieldDescription(
"Whether or not the publish message should be retained by the messaging engine. "
+ "When a subscriber connects, it gets the latest retained message. "
+ "Defaults to `False`, which will clear the retained message from the server.")
@Nullable
public abstract Boolean getRetained();

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setConnectionConfiguration(ConnectionConfiguration foo);

public abstract Builder setRetained(Boolean retained);

public abstract WriteConfiguration build();
}
}

@Override
public String identifier() {
return "beam:schematransform:org.apache.beam:mqtt_write:v1";
}

@Override
protected SchemaTransform from(WriteConfiguration configuration) {
return new MqttWriteSchemaTransform(configuration);
}

private static class MqttWriteSchemaTransform extends SchemaTransform {
private final WriteConfiguration config;

MqttWriteSchemaTransform(WriteConfiguration configuration) {
this.config = configuration;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
PCollection<Row> inputRows = input.getSinglePCollection();

Preconditions.checkState(
inputRows.getSchema().getFieldCount() == 1
&& inputRows.getSchema().getField(0).getType().equals(Schema.FieldType.BYTES),
"Expected only one Schema field containing bytes, but instead received: %s",
inputRows.getSchema());

MqttIO.Write writeTransform =
MqttIO.write().withConnectionConfiguration(config.getConnectionConfiguration());
Boolean retained = config.getRetained();
if (retained != null) {
writeTransform = writeTransform.withRetained(retained);
}

inputRows
.apply(
"Extract bytes",
ParDo.of(
new DoFn<Row, byte[]>() {
@ProcessElement
public void processElement(
@Element Row row, OutputReceiver<byte[]> outputReceiver) {
outputReceiver.output(
org.apache.beam.sdk.util.Preconditions.checkStateNotNull(
row.getBytes(0)));
}
}))
.apply(writeTransform);

return PCollectionRowTuple.empty(inputRows.getPipeline());
}
}
}
Loading
Loading