|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.sdk.io.mqtt; |
| 19 | + |
| 20 | +import static org.apache.beam.sdk.io.mqtt.MqttIO.ConnectionConfiguration; |
| 21 | +import static org.apache.beam.sdk.io.mqtt.MqttReadSchemaTransformProvider.ReadConfiguration; |
| 22 | + |
| 23 | +import com.google.auto.service.AutoService; |
| 24 | +import com.google.auto.value.AutoValue; |
| 25 | +import java.io.Serializable; |
| 26 | +import javax.annotation.Nullable; |
| 27 | +import org.apache.beam.sdk.schemas.AutoValueSchema; |
| 28 | +import org.apache.beam.sdk.schemas.Schema; |
| 29 | +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; |
| 30 | +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; |
| 31 | +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; |
| 32 | +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; |
| 33 | +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; |
| 34 | +import org.apache.beam.sdk.transforms.DoFn; |
| 35 | +import org.apache.beam.sdk.transforms.ParDo; |
| 36 | +import org.apache.beam.sdk.values.PCollection; |
| 37 | +import org.apache.beam.sdk.values.PCollectionRowTuple; |
| 38 | +import org.apache.beam.sdk.values.Row; |
| 39 | +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; |
| 40 | +import org.joda.time.Duration; |
| 41 | + |
| 42 | +@AutoService(SchemaTransformProvider.class) |
| 43 | +public class MqttReadSchemaTransformProvider |
| 44 | + extends TypedSchemaTransformProvider<ReadConfiguration> { |
| 45 | + @DefaultSchema(AutoValueSchema.class) |
| 46 | + @AutoValue |
| 47 | + public abstract static class ReadConfiguration implements Serializable { |
| 48 | + public static Builder builder() { |
| 49 | + return new AutoValue_MqttReadSchemaTransformProvider_ReadConfiguration.Builder(); |
| 50 | + } |
| 51 | + |
| 52 | + @SchemaFieldDescription("Configuration options to set up the MQTT connection.") |
| 53 | + public abstract ConnectionConfiguration getConnectionConfiguration(); |
| 54 | + |
| 55 | + @SchemaFieldDescription( |
| 56 | + "The max number of records to receive. Setting this will result in a bounded PCollection.") |
| 57 | + @Nullable |
| 58 | + public abstract Long getMaxNumRecords(); |
| 59 | + |
| 60 | + @SchemaFieldDescription( |
| 61 | + "The maximum time for this source to read messages. Setting this will result in a bounded PCollection.") |
| 62 | + @Nullable |
| 63 | + public abstract Long getMaxReadTimeSeconds(); |
| 64 | + |
| 65 | + @AutoValue.Builder |
| 66 | + public abstract static class Builder { |
| 67 | + public abstract Builder setConnectionConfiguration( |
| 68 | + ConnectionConfiguration connectionConfiguration); |
| 69 | + |
| 70 | + public abstract Builder setMaxNumRecords(Long maxNumRecords); |
| 71 | + |
| 72 | + public abstract Builder setMaxReadTimeSeconds(Long maxReadTimeSeconds); |
| 73 | + |
| 74 | + public abstract ReadConfiguration build(); |
| 75 | + } |
| 76 | + } |
| 77 | + |
| 78 | + @Override |
| 79 | + public String identifier() { |
| 80 | + return "beam:schematransform:org.apache.beam:mqtt_read:v1"; |
| 81 | + } |
| 82 | + |
| 83 | + @Override |
| 84 | + protected SchemaTransform from(ReadConfiguration configuration) { |
| 85 | + return new MqttReadSchemaTransform(configuration); |
| 86 | + } |
| 87 | + |
| 88 | + private static class MqttReadSchemaTransform extends SchemaTransform { |
| 89 | + private final ReadConfiguration config; |
| 90 | + |
| 91 | + MqttReadSchemaTransform(ReadConfiguration configuration) { |
| 92 | + this.config = configuration; |
| 93 | + } |
| 94 | + |
| 95 | + @Override |
| 96 | + public PCollectionRowTuple expand(PCollectionRowTuple input) { |
| 97 | + Preconditions.checkState( |
| 98 | + input.getAll().isEmpty(), |
| 99 | + "Expected zero input PCollections for this source, but found: %", |
| 100 | + input.getAll().keySet()); |
| 101 | + |
| 102 | + MqttIO.Read<byte[]> readTransform = |
| 103 | + MqttIO.read().withConnectionConfiguration(config.getConnectionConfiguration()); |
| 104 | + |
| 105 | + Long maxRecords = config.getMaxNumRecords(); |
| 106 | + Long maxReadTime = config.getMaxReadTimeSeconds(); |
| 107 | + if (maxRecords != null) { |
| 108 | + readTransform = readTransform.withMaxNumRecords(maxRecords); |
| 109 | + } |
| 110 | + if (maxReadTime != null) { |
| 111 | + readTransform = readTransform.withMaxReadTime(Duration.standardSeconds(maxReadTime)); |
| 112 | + } |
| 113 | + |
| 114 | + Schema outputSchema = Schema.builder().addByteArrayField("bytes").build(); |
| 115 | + |
| 116 | + PCollection<Row> outputRows = |
| 117 | + input |
| 118 | + .getPipeline() |
| 119 | + .apply(readTransform) |
| 120 | + .apply( |
| 121 | + "Wrap in Beam Rows", |
| 122 | + ParDo.of( |
| 123 | + new DoFn<byte[], Row>() { |
| 124 | + @ProcessElement |
| 125 | + public void processElement( |
| 126 | + @Element byte[] data, OutputReceiver<Row> outputReceiver) { |
| 127 | + outputReceiver.output( |
| 128 | + Row.withSchema(outputSchema).addValue(data).build()); |
| 129 | + } |
| 130 | + })) |
| 131 | + .setRowSchema(outputSchema); |
| 132 | + |
| 133 | + return PCollectionRowTuple.of("output", outputRows); |
| 134 | + } |
| 135 | + } |
| 136 | +} |
0 commit comments