-
Notifications
You must be signed in to change notification settings - Fork 4.6k
[mqtt] Add portable MqttIO Read/Write transforms for batch and streaming (revives #32385) #38493
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
Merged
Merged
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,52 @@ | ||
| /* | ||
| * 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. | ||
| */ | ||
|
|
||
| apply plugin: 'org.apache.beam.module' | ||
| apply plugin: 'application' | ||
| mainClassName = "org.apache.beam.sdk.expansion.service.ExpansionService" | ||
|
|
||
| applyJavaNature( | ||
| automaticModuleName: 'org.apache.beam.sdk.io.messaging.expansion.service', | ||
| exportJavadoc: false, | ||
| validateShadowJar: false, | ||
| shadowClosure: {}, | ||
| ) | ||
|
|
||
| shadowJar { | ||
| manifest { | ||
| attributes(["Multi-Release": true]) | ||
| } | ||
| mergeServiceFiles() | ||
| } | ||
|
|
||
| description = "Apache Beam :: SDKs :: Java :: IO :: Messaging Expansion Service" | ||
| ext.summary = "Expansion service serving messaging IOs (e.g. MQTT)" | ||
|
|
||
| dependencies { | ||
| implementation project(":sdks:java:expansion-service") | ||
| permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 | ||
| implementation project(":sdks:java:io:mqtt") | ||
| permitUnusedDeclared project(":sdks:java:io:mqtt") // BEAM-11761 | ||
| runtimeOnly library.java.slf4j_jdk14 | ||
| } | ||
|
|
||
| task runExpansionService (type: JavaExec) { | ||
| mainClass = "org.apache.beam.sdk.expansion.service.ExpansionService" | ||
| classpath = sourceSets.test.runtimeClasspath | ||
| args = [project.findProperty("constructionService.port") ?: "8097"] | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
150 changes: 150 additions & 0 deletions
150
...va/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttReadSchemaTransformProvider.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,150 @@ | ||
| /* | ||
| * 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 | ||
| public String description() { | ||
| return "Reads messages from an MQTT broker and outputs each payload as a single `bytes` " | ||
| + "field.\n" | ||
| + "\n" | ||
| + "By default the read is unbounded (streaming): it keeps consuming messages from the " | ||
| + "subscribed topic until the pipeline is stopped. Setting `maxNumRecords` and/or " | ||
| + "`maxReadTimeSeconds` bounds the read, producing a bounded (batch) PCollection.\n" | ||
| + "\n" | ||
| + "Note: streaming reads require a runner that supports portable streaming (e.g. Prism, " | ||
| + "Flink, or Dataflow). The legacy local Python DirectRunner does not execute portable " | ||
| + "streaming cross-language reads."; | ||
| } | ||
|
|
||
| @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: %s", | ||
| input.getAll().keySet()); | ||
|
|
||
| MqttIO.Read<byte[]> 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); | ||
| } | ||
| } | ||
| } | ||
132 changes: 132 additions & 0 deletions
132
...a/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttWriteSchemaTransformProvider.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,132 @@ | ||
| /* | ||
| * 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 connectionConfiguration); | ||
|
|
||
| public abstract Builder setRetained(Boolean retained); | ||
|
|
||
| public abstract WriteConfiguration build(); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public String identifier() { | ||
| return "beam:schematransform:org.apache.beam:mqtt_write:v1"; | ||
| } | ||
|
|
||
| @Override | ||
| public String description() { | ||
| return "Publishes messages to an MQTT broker. Expects an input PCollection of rows with a " | ||
| + "single `bytes` field, each of which is published as one MQTT message.\n" | ||
| + "\n" | ||
| + "Works with both bounded (batch) and unbounded (streaming) input PCollections."; | ||
| } | ||
|
|
||
| @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<byte[]> 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()); | ||
| } | ||
| } | ||
| } |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.