From 9fdc59b2c0f9a54457d9cde3117e9f0c104d59f1 Mon Sep 17 00:00:00 2001 From: Damon Date: Wed, 18 Oct 2023 14:01:02 -0700 Subject: [PATCH] [RRIO] [Throttle] stub the dequeue transform (#29056) --- .../io/requestresponse/ThrottleDequeue.java | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java new file mode 100644 index 0000000000000..085b13b5e1120 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/ThrottleDequeue.java @@ -0,0 +1,101 @@ +/* + * 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.io.requestresponse; + +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.io.requestresponse.ThrottleDequeue.Result; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Instant; + +/** + * {@link ThrottleDequeue} dequeues {@link RequestT} elements at a fixed rate yielding a {@link + * Result} containing the dequeued {@link RequestT} {@link PCollection} and a {@link ApiIOError} + * {@link PCollection} of any errors. + */ +class ThrottleDequeue extends PTransform, Result> { + + private static final TupleTag FAILURE_TAG = new TupleTag() {}; + + // TODO(damondouglas): remove suppress warnings after instance utilized. + @SuppressWarnings({"unused"}) + private final Configuration configuration; + + private ThrottleDequeue(Configuration configuration) { + this.configuration = configuration; + } + + @Override + public Result expand(PCollection input) { + // TODO(damondouglas): expand in a future PR. + return new Result<>(new TupleTag() {}, PCollectionTuple.empty(input.getPipeline())); + } + + @AutoValue + abstract static class Configuration { + + @AutoValue.Builder + abstract static class Builder { + abstract Configuration build(); + } + } + + /** The {@link Result} of dequeuing {@link RequestT}s. */ + static class Result implements POutput { + + static Result of(TupleTag requestsTag, PCollectionTuple pct) { + return new Result<>(requestsTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag requestsTag; + private final PCollection requests; + private final PCollection failures; + + private Result(TupleTag requestsTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.requestsTag = requestsTag; + this.requests = pct.get(requestsTag); + this.failures = pct.get(FAILURE_TAG); + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of( + requestsTag, requests, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + } +}