Ganeshsivakumar commented on code in PR #36623: URL: https://github.com/apache/beam/pull/36623#discussion_r2549811042
########## sdks/java/ml/inference/remote/src/main/java/org/apache/beam/sdk/ml/inference/remote/RemoteInference.java: ########## @@ -0,0 +1,169 @@ +/* + * 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.ml.remoteinference; + +import org.apache.beam.sdk.ml.remoteinference.base.*; +import org.apache.beam.sdk.transforms.*; +import org.checkerframework.checker.nullness.qual.Nullable; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.sdk.values.PCollection; +import com.google.auto.value.AutoValue; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * A {@link PTransform} for making remote inference calls to external machine learning services. + * + * <p>{@code RemoteInference} provides a framework for integrating remote ML model + * inference into Apache Beam pipelines and handles the communication between pipelines + * and external inference APIs. + * + * <h3>Example: OpenAI Model Inference</h3> + * + * <pre>{@code + * // Create model parameters + * OpenAIModelParameters params = OpenAIModelParameters.builder() + * .apiKey("your-api-key") + * .modelName("gpt-4") + * .instructionPrompt("Analyse sentiment as positive or negative") + * .build(); + * + * // Apply remote inference transform + * PCollection<OpenAIModelInput> inputs = pipeline.apply(Create.of( + * OpenAIModelInput.create("An excellent B2B SaaS solution that streamlines business processes efficiently."), + * OpenAIModelInput.create("Really impressed with the innovative features!") + * )); + * + * PCollection<Iterable<PredictionResult<OpenAIModelInput, OpenAIModelResponse>>> results = + * inputs.apply( + * RemoteInference.<OpenAIModelInput, OpenAIModelResponse>invoke() + * .handler(OpenAIModelHandler.class) + * .withParameters(params) + * ); + * }</pre> + * + */ +@SuppressWarnings({ "rawtypes", "unchecked" }) +public class RemoteInference { + + /** Invoke the model handler with model parameters */ + public static <InputT extends BaseInput, OutputT extends BaseResponse> Invoke<InputT, OutputT> invoke() { + return new AutoValue_RemoteInference_Invoke.Builder<InputT, OutputT>().setParameters(null) + .build(); + } + + private RemoteInference() { + } + + @AutoValue + public abstract static class Invoke<InputT extends BaseInput, OutputT extends BaseResponse> + extends PTransform<PCollection<InputT>, PCollection<Iterable<PredictionResult<InputT, OutputT>>>> { + + abstract @Nullable Class<? extends BaseModelHandler> handler(); + + abstract @Nullable BaseModelParameters parameters(); + + + abstract Builder<InputT, OutputT> builder(); + + @AutoValue.Builder + abstract static class Builder<InputT extends BaseInput, OutputT extends BaseResponse> { + + abstract Builder<InputT, OutputT> setHandler(Class<? extends BaseModelHandler> modelHandler); + + abstract Builder<InputT, OutputT> setParameters(BaseModelParameters modelParameters); + + + abstract Invoke<InputT, OutputT> build(); + } + + /** + * Model handler class for inference. + */ + public Invoke<InputT, OutputT> handler(Class<? extends BaseModelHandler> modelHandler) { + return builder().setHandler(modelHandler).build(); + } + + /** + * Configures the parameters for model initialization. + */ + public Invoke<InputT, OutputT> withParameters(BaseModelParameters modelParameters) { + return builder().setParameters(modelParameters).build(); + } + + + @Override + public PCollection<Iterable<PredictionResult<InputT, OutputT>>> expand(PCollection<InputT> input) { + checkArgument(handler() != null, "handler() is required"); + checkArgument(parameters() != null, "withParameters() is required"); + return input + .apply("WrapInputInList", MapElements.via(new SimpleFunction<InputT, List<InputT>>() { + @Override + public List<InputT> apply(InputT element) { + return Collections.singletonList(element); + } + })) + // Pass the list to the inference function + .apply("RemoteInference", ParDo.of(new RemoteInferenceFn<InputT, OutputT>(this))); Review Comment: We have plan to implement Java `BatchElements` transform to group elements in remote inference. In a separate PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
