From 964710f4f9d31cc8493a633e823ea1b3500faead Mon Sep 17 00:00:00 2001 From: Damon Date: Wed, 18 Oct 2023 14:00:26 -0700 Subject: [PATCH] [RRIO] [Call] stub the Call transform (#29060) --- .../apache/beam/io/requestresponse/Call.java | 116 ++++++++++++++++++ 1 file changed, 116 insertions(+) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java new file mode 100644 index 000000000000..4f854ea69c7e --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponse/Call.java @@ -0,0 +1,116 @@ +/* + * 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.Call.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; + +/** + * {@link Call} transforms a {@link RequestT} {@link PCollection} into a {@link ResponseT} {@link + * PCollection} and {@link ApiIOError} {@link PCollection}, both wrapped in a {@link Result}. + */ +class Call extends PTransform, Result> { + + private static final TupleTag FAILURE_TAG = new TupleTag() {}; + + // TODO(damondouglas): remove suppress warnings when configuration utilized in future PR. + @SuppressWarnings({"unused"}) + private final Configuration configuration; + + private Call(Configuration configuration) { + this.configuration = configuration; + } + + /** Configuration details for {@link Call}. */ + @AutoValue + abstract static class Configuration { + + static Builder builder() { + return new AutoValue_Call_Configuration.Builder<>(); + } + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Configuration build(); + } + } + + @Override + public Result expand(PCollection input) { + return Result.of(new TupleTag() {}, PCollectionTuple.empty(input.getPipeline())); + } + + /** + * The {@link Result} of processing request {@link PCollection} into response {@link PCollection}. + */ + static class Result implements POutput { + + static Result of(TupleTag responseTag, PCollectionTuple pct) { + return new Result<>(responseTag, pct); + } + + private final Pipeline pipeline; + private final TupleTag responseTag; + private final PCollection responses; + private final PCollection failures; + + private Result(TupleTag responseTag, PCollectionTuple pct) { + this.pipeline = pct.getPipeline(); + this.responseTag = responseTag; + this.responses = pct.get(responseTag); + this.failures = pct.get(FAILURE_TAG); + } + + public PCollection getResponses() { + return responses; + } + + public PCollection getFailures() { + return failures; + } + + @Override + public Pipeline getPipeline() { + return this.pipeline; + } + + @Override + public Map, PValue> expand() { + return ImmutableMap.of( + responseTag, responses, + FAILURE_TAG, failures); + } + + @Override + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} + } +}