Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
32 changes: 32 additions & 0 deletions sdks/java/ml/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.
*/

plugins { id 'org.apache.beam.module' }
applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.ml',
)
provideIntegrationTestingDependencies()
enableJavaPerformanceTesting()

description = "Apache Beam :: SDKs :: Java :: ML"
ext.summary = "Java ML module"

dependencies {


}
21 changes: 21 additions & 0 deletions sdks/java/ml/remoteinference/build.gradle.kts
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
plugins {
id("org.apache.beam.module")
id("java-library")
}

description = "Apache Beam :: SDKs :: Java :: ML :: RemoteInference"

dependencies {
// Core Beam SDK
implementation(project(":sdks:java:core"))

implementation("com.openai:openai-java:4.3.0")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would it make more sense to scope the Open AI dependency for the Open AI model handler specifically? Otherwise any usage of a remote model handler, regardless of target service, would have a bunch of extra dependencies.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Totally, I thought about as well, model handlers can be their own separate modules.

implementation("com.google.auto.value:auto-value:1.11.0")
implementation("com.google.auto.value:auto-value-annotations:1.11.0")

// testing
testImplementation(project(":runners:direct-java"))
testImplementation("junit:junit:4.13.2")
testImplementation(project(":sdks:java:testing:test-utils"))
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
package org.apache.beam.sdk.ml.remoteinference;

import org.checkerframework.checker.nullness.qual.Nullable;

import org.apache.beam.sdk.ml.remoteinference.base.BaseInput;
import org.apache.beam.sdk.ml.remoteinference.base.BaseModelHandler;
import org.apache.beam.sdk.ml.remoteinference.base.BaseModelParameters;
import org.apache.beam.sdk.ml.remoteinference.base.BaseResponse;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;


import com.google.auto.value.AutoValue;

@SuppressWarnings({ "rawtypes", "unchecked" })
public class RemoteInference {

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<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();
}

public Invoke<InputT, OutputT> handler(Class<? extends BaseModelHandler> modelHandler) {
return builder().setHandler(modelHandler).build();
}

public Invoke<InputT, OutputT> withParameters(BaseModelParameters modelParameters) {
return builder().setParameters(modelParameters).build();
}

@Override
public PCollection<OutputT> expand(PCollection<InputT> input) {
return input.apply(ParDo.of(new RemoteInferenceFn<>(this)));
}

static class RemoteInferenceFn<InputT extends BaseInput, OutputT extends BaseResponse>
extends DoFn<InputT, OutputT> {

private final Class<? extends BaseModelHandler> handlerClass;
private final BaseModelParameters parameters;
private transient BaseModelHandler handler;

RemoteInferenceFn(Invoke<InputT, OutputT> spec) {
this.handlerClass = spec.handler();
this.parameters = spec.parameters();
}

@Setup
public void setupHandler() {
try {
this.handler = handlerClass.getDeclaredConstructor().newInstance();
this.handler.createClient(parameters);
} catch (Exception e) {
throw new RuntimeException("Failed to instantiate handler: "
+ handlerClass.getName(), e);
}
}

@ProcessElement
public void processElement(ProcessContext c) {
OutputT response = (OutputT) this.handler.request(c.element());
c.output(response);
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
package org.apache.beam.sdk.ml.remoteinference.base;

import java.io.Serializable;

public abstract class BaseInput implements Serializable {

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,11 @@
package org.apache.beam.sdk.ml.remoteinference.base;

public interface BaseModelHandler<ParamT extends BaseModelParameters, InputT extends BaseInput, OutputT extends BaseResponse> {

// initialize the model with provided parameters
public void createClient(ParamT parameters);

// Logic to invoke model provider
public OutputT request(InputT input);

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
package org.apache.beam.sdk.ml.remoteinference.base;

import java.io.Serializable;

public interface BaseModelParameters extends Serializable {

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
package org.apache.beam.sdk.ml.remoteinference.base;

import java.io.Serializable;

public abstract class BaseResponse implements Serializable {

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
package org.apache.beam.sdk.ml.remoteinference.openai;

import com.openai.client.OpenAIClient;
import com.openai.client.okhttp.OpenAIOkHttpClient;
import com.openai.models.responses.ResponseCreateParams;
import org.apache.beam.sdk.ml.remoteinference.base.BaseModelHandler;

import java.util.stream.Collectors;

public class OpenAiModelHandler
implements BaseModelHandler<OpenAiModelParameters, OpenAiModelInput, OpenAiModelResponse> {

private transient OpenAIClient client;
private transient ResponseCreateParams clientParams;
private OpenAiModelParameters modelParameters;

@Override
public void createClient(OpenAiModelParameters parameters) {
this.modelParameters = parameters;
this.client = OpenAIOkHttpClient.builder()
.apiKey(this.modelParameters.getApiKey())
.build();
}

@Override
public OpenAiModelResponse request(OpenAiModelInput input) {

this.clientParams = ResponseCreateParams.builder()
.model(this.modelParameters.getModelName())
.input(input.getInput())
.build();

String output = client.responses().create(clientParams).output().stream()
.flatMap(item -> item.message().stream())
.flatMap(message -> message.content().stream())
.flatMap(content -> content.outputText().stream())
.map(outputText -> outputText.text())
.collect(Collectors.joining());

OpenAiModelResponse res = OpenAiModelResponse.create(input.getInput(), output);
return res;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
package org.apache.beam.sdk.ml.remoteinference.openai;

import org.apache.beam.sdk.ml.remoteinference.base.BaseInput;

public class OpenAiModelInput extends BaseInput {

private final String input;

private OpenAiModelInput(String input) {

this.input = input;
}

public String getInput() {
return input;
}

public static OpenAiModelInput create(String input) {
return new OpenAiModelInput(input);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
package org.apache.beam.sdk.ml.remoteinference.openai;

import org.apache.beam.sdk.ml.remoteinference.base.BaseModelParameters;

public class OpenAiModelParameters implements BaseModelParameters {

private final String apiKey;
private final String modelName;
private final String instructionPrompt;

private OpenAiModelParameters(Builder builder) {
this.apiKey = builder.apiKey;
this.modelName = builder.modelName;
this.instructionPrompt = builder.instructionPrompt;
}

public String getApiKey() {
return apiKey;
}

public String getModelName() {
return modelName;
}

public String getInstructionPrompt() {
return instructionPrompt;
}

public static Builder builder() {
return new Builder();
}

public static class Builder {
private String apiKey;
private String modelName;
private String instructionPrompt;

private Builder() {
}

public Builder apiKey(String apiKey) {
this.apiKey = apiKey;
return this;
}

public Builder modelName(String modelName) {
this.modelName = modelName;
return this;
}

public Builder instructionPrompt(String prompt) {
this.instructionPrompt = prompt;
return this;
}

public OpenAiModelParameters build() {
return new OpenAiModelParameters(this);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
package org.apache.beam.sdk.ml.remoteinference.openai;

import org.apache.beam.sdk.ml.remoteinference.base.BaseResponse;

public class OpenAiModelResponse extends BaseResponse {

private final String input;
private final String output;

private OpenAiModelResponse(String input, String output) {
this.input = input;
this.output = output;
}

public String getInput() {
return input;
}

public String getOutput() {
return output;
}

public static OpenAiModelResponse create(String input, String output) {
return new OpenAiModelResponse(input, output);
}
}
43 changes: 43 additions & 0 deletions sdks/java/ml/remoteinference/src/test/java/Example.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
import org.apache.beam.runners.direct.DirectRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.ml.remoteinference.RemoteInference;
import org.apache.beam.sdk.ml.remoteinference.openai.OpenAiModelHandler;
import org.apache.beam.sdk.ml.remoteinference.openai.OpenAiModelInput;
import org.apache.beam.sdk.ml.remoteinference.openai.OpenAiModelParameters;
import org.apache.beam.sdk.ml.remoteinference.openai.OpenAiModelResponse;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.TypeDescriptor;

public class Example {
public static void main(String[] args) {

PipelineOptions options = PipelineOptionsFactory.create();
options.setRunner(DirectRunner.class);
Pipeline p = Pipeline.create(options);

p.apply("text", Create.of(
"An excellent B2B SaaS solution that streamlines business processes efficiently. The platform is user-friendly and highly reliable. Overall, it delivers great value for enterprise teams."))
.apply(MapElements.into(TypeDescriptor.of(OpenAiModelInput.class))
.via(OpenAiModelInput::create))
.apply("inference", RemoteInference.<OpenAiModelInput, OpenAiModelResponse>invoke()
.handler(OpenAiModelHandler.class)
.withParameters(OpenAiModelParameters.builder()
.apiKey("key")
.modelName("gpt-5-mini")
.instructionPrompt("Analyse sentiment as positive or negative")
.build()))
.apply("print output", ParDo.of(new DoFn<OpenAiModelResponse, Void>() {
@ProcessElement
public void print(ProcessContext c) {
System.out.println("OUTPUT: " + c.element().getOutput());
}
}));

p.run();
}
}
2 changes: 2 additions & 0 deletions settings.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -383,3 +383,5 @@ include("sdks:java:extensions:sql:iceberg")
findProject(":sdks:java:extensions:sql:iceberg")?.name = "iceberg"
include("examples:java:iceberg")
findProject(":examples:java:iceberg")?.name = "iceberg"

include("sdks:java:ml:remoteinference")
Loading