diff --git a/sdks/java/extensions/python/build.gradle b/sdks/java/extensions/python/build.gradle index 2c3f68c649452..7a04d953ccb5f 100644 --- a/sdks/java/extensions/python/build.gradle +++ b/sdks/java/extensions/python/build.gradle @@ -30,6 +30,7 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":runners:core-construction-java") implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(path: ":sdks:java:transform-service:launcher", configuration: "shadow") testImplementation library.java.junit testImplementation library.java.hamcrest testImplementation project(":runners:core-construction-java").sourceSets.test.output diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index 8e27901efb1e0..7a217504d626f 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.JavaFieldSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; @@ -45,8 +46,10 @@ import org.apache.beam.sdk.schemas.utils.StaticSchemaInference; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.PythonCallableSource; +import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -447,30 +450,64 @@ public OutputT expand(InputT input) { 15000); return apply(input, expansionService, payload); } else { + OutputT output = null; int port = PythonService.findAvailablePort(); - ImmutableList.Builder args = ImmutableList.builder(); - args.add("--port=" + port, "--fully_qualified_name_glob=*", "--pickle_library=cloudpickle"); - if (!extraPackages.isEmpty()) { - File requirementsFile = File.createTempFile("requirements", ".txt"); - requirementsFile.deleteOnExit(); - try (Writer fout = - new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { - for (String pkg : extraPackages) { - fout.write(pkg); - fout.write('\n'); + PipelineOptionsFactory.register(PythonExternalTransformOptions.class); + if (input + .getPipeline() + .getOptions() + .as(PythonExternalTransformOptions.class) + .getUseTransformService()) { + // A unique project name ensures that this expansion gets a dedicated instance of the + // transform service. + String projectName = UUID.randomUUID().toString(); + TransformServiceLauncher service = TransformServiceLauncher.forProject(projectName, port); + service.setBeamVersion(ReleaseInfo.getReleaseInfo().getSdkVersion()); + // TODO(https://github.com/apache/beam/issues/26833): add support for installing extra + // packages. + if (!extraPackages.isEmpty()) { + throw new RuntimeException( + "Transform Service does not support installing extra packages yet"); + } + try { + // Starting the transform service. + service.start(); + // Waiting the service to be ready. + service.waitTillUp(15000); + // Expanding the transform. + output = apply(input, String.format("localhost:%s", port), payload); + } finally { + // Shutting down the transform service. + service.shutdown(); + } + return output; + } else { + + ImmutableList.Builder args = ImmutableList.builder(); + args.add( + "--port=" + port, "--fully_qualified_name_glob=*", "--pickle_library=cloudpickle"); + if (!extraPackages.isEmpty()) { + File requirementsFile = File.createTempFile("requirements", ".txt"); + requirementsFile.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + for (String pkg : extraPackages) { + fout.write(pkg); + fout.write('\n'); + } } + args.add("--requirements_file=" + requirementsFile.getAbsolutePath()); + } + PythonService service = + new PythonService( + "apache_beam.runners.portability.expansion_service_main", args.build()) + .withExtraPackages(extraPackages); + try (AutoCloseable p = service.start()) { + // allow more time waiting for the port ready for transient expansion service setup. + PythonService.waitForPort("localhost", port, 60000); + return apply(input, String.format("localhost:%s", port), payload); } - args.add("--requirements_file=" + requirementsFile.getAbsolutePath()); - } - PythonService service = - new PythonService( - "apache_beam.runners.portability.expansion_service_main", args.build()) - .withExtraPackages(extraPackages); - try (AutoCloseable p = service.start()) { - // allow more time waiting for the port ready for transient expansion service setup. - PythonService.waitForPort("localhost", port, 60000); - return apply(input, String.format("localhost:%s", port), payload); } } } catch (RuntimeException exn) { diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransformOptions.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransformOptions.java new file mode 100644 index 0000000000000..0c8c7859349f9 --- /dev/null +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransformOptions.java @@ -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. + */ +package org.apache.beam.sdk.extensions.python; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; + +/** Pipeline options for {@link PythonExternalTransform}. */ +public interface PythonExternalTransformOptions extends PipelineOptions { + + @Description("Use Docker Compose based Beam Transform Service to expand transforms.") + @Default.Boolean(false) + boolean getUseTransformService(); + + void setUseTransformService(boolean useTransformService); +} diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransformOptionsRegistrar.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransformOptionsRegistrar.java new file mode 100644 index 0000000000000..a7d138b409d74 --- /dev/null +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransformOptionsRegistrar.java @@ -0,0 +1,37 @@ +/* + * 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.extensions.python; + +import com.google.auto.service.AutoService; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; + +/** A registrar for {@link PythonExternalTransformOptions}. */ +@AutoService(PipelineOptionsRegistrar.class) +@Internal +public class PythonExternalTransformOptionsRegistrar implements PipelineOptionsRegistrar { + + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.>builder() + .add(PythonExternalTransformOptions.class) + .build(); + } +} diff --git a/sdks/java/transform-service/build.gradle b/sdks/java/transform-service/build.gradle index 7bd6b5917df43..94c1d85f27d5b 100644 --- a/sdks/java/transform-service/build.gradle +++ b/sdks/java/transform-service/build.gradle @@ -16,20 +16,18 @@ * limitations under the License. */ -plugins { - id 'org.apache.beam.module' - id 'com.github.johnrengelman.shadow' -} - apply plugin: 'org.apache.beam.module' + applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.transform.service', + exportJavadoc: false, + validateShadowJar: false, + shadowClosure: {}, ) description = "Apache Beam :: SDKs :: Java :: Transform Service" -ext.summary = """Contains code that can be used to run an transform service.""" - +ext.summary = """Contains core code of the transform service.""" // Exclude tests that need a runner test { diff --git a/sdks/java/transform-service/docker-compose/.env b/sdks/java/transform-service/docker-compose/.env index 78a7b607c3fac..5de5982cfa301 100644 --- a/sdks/java/transform-service/docker-compose/.env +++ b/sdks/java/transform-service/docker-compose/.env @@ -13,3 +13,5 @@ BEAM_VERSION=$BEAM_VERSION CREDENTIALS_VOLUME=$CREDENTIALS_VOLUME GOOGLE_APPLICATION_CREDENTIALS_FILE_NAME=application_default_credentials.json +COMPOSE_PROJECT_NAME=apache.beam.transform.service +TRANSFORM_SERVICE_PORT=$TRANSFORM_SERVICE_PORT diff --git a/sdks/java/transform-service/docker-compose/docker-compose.yml b/sdks/java/transform-service/docker-compose/docker-compose.yml index 336047520d52c..c0a28f6ae07a1 100644 --- a/sdks/java/transform-service/docker-compose/docker-compose.yml +++ b/sdks/java/transform-service/docker-compose/docker-compose.yml @@ -17,7 +17,7 @@ services: restart: on-failure command: -port 5001 ports: - - "5001:5001" + - "${TRANSFORM_SERVICE_PORT}:5001" expansion-service-1: image: "apache/beam_java_expansion_service:${BEAM_VERSION}" restart: on-failure diff --git a/sdks/java/transform-service/launcher/build.gradle b/sdks/java/transform-service/launcher/build.gradle new file mode 100644 index 0000000000000..acfb3e3081735 --- /dev/null +++ b/sdks/java/transform-service/launcher/build.gradle @@ -0,0 +1,62 @@ +/* + * 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. + */ +apply plugin: 'org.apache.beam.module' +apply plugin: 'application' +mainClassName = "org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher" + +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.transform.service', + exportJavadoc: false, + validateShadowJar: false, + shadowClosure: {}, +) + + +description = "Apache Beam :: SDKs :: Java :: Transform Service :: Launcher" +ext.summary = """Contains code that can be used to run an transform service.""" + + +// Exclude tests that need a runner +test { + systemProperty "beamUseDummyRunner", "true" + useJUnit { + excludeCategories "org.apache.beam.sdk.testing.NeedsRunner" + } +} + +dependencies { + shadow library.java.vendored_guava_26_0_jre + shadow library.java.slf4j_api + shadow library.java.args4j + runtimeOnly library.java.slf4j_jdk14 +} + +sourceSets { + main { + resources { + srcDirs "../docker-compose" + } + output.resourcesDir = "$buildDir/resources/docker-compose/$name" + } +} + +jar { + manifest { + attributes 'Main-Class': application.mainClass + } +} diff --git a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java new file mode 100644 index 0000000000000..bca6cc46bf949 --- /dev/null +++ b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java @@ -0,0 +1,306 @@ +/* + * 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.transformservice.launcher; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.TimeoutException; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.Files; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.kohsuke.args4j.CmdLineException; +import org.kohsuke.args4j.CmdLineParser; +import org.kohsuke.args4j.Option; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A utility that can be used to manage a Beam Transform Service. + * + *

Can be either used programatically or as an executable jar. + */ +public class TransformServiceLauncher { + + private static final Logger LOG = LoggerFactory.getLogger(TransformServiceLauncher.class); + + private static final String DEFAULT_PROJECT_NAME = "apache.beam.transform.service"; + + private static final String COMMAND_POSSIBLE_VALUES = "\"up\", \"down\" and \"ps\""; + + private static Map launchers = new HashMap<>(); + + private List dockerComposeStartCommandPrefix = new ArrayList<>(); + + private Map environmentVariables = new HashMap<>(); + + // Amount of time (in milliseconds) to wait till the Docker Compose starts up. + private static final int DEFAULT_START_WAIT_TIME = 25000; + private static final int STATUS_LOGGER_WAIT_TIME = 3000; + + @SuppressWarnings("argument") + private TransformServiceLauncher(@Nullable String projectName, int port) throws IOException { + LOG.info("Initializing the Beam Transform Service {}.", projectName); + + String tmpDirLocation = System.getProperty("java.io.tmpdir"); + // We use Docker Compose project name as the name of the temporary directory to isolate + // different transform service instances that may be running in the same machine. + Path tmpDirPath = Paths.get(tmpDirLocation, projectName); + java.nio.file.Files.createDirectories(tmpDirPath); + + String tmpDir = tmpDirPath.toFile().getAbsolutePath(); + + File dockerComposeFile = Paths.get(tmpDir, "docker-compose.yml").toFile(); + try (FileOutputStream fout = new FileOutputStream(dockerComposeFile)) { + ByteStreams.copy(getClass().getResourceAsStream("/docker-compose.yml"), fout); + } + + File envFile = Paths.get(tmpDir, ".env").toFile(); + try (FileOutputStream fout = new FileOutputStream(envFile)) { + ByteStreams.copy(getClass().getResourceAsStream("/.env"), fout); + } + + File credentialsDir = Paths.get(tmpDir, "credentials_dir").toFile(); + LOG.info( + "Creating a temporary directory for storing credentials: " + + credentialsDir.getAbsolutePath()); + + if (credentialsDir.exists()) { + LOG.info("Reusing the existing credentials directory " + credentialsDir.getAbsolutePath()); + } else { + if (!credentialsDir.mkdir()) { + throw new IOException( + "Could not create a temporary directory for storing credentials: " + + credentialsDir.getAbsolutePath()); + } + + LOG.info("Copying the Google Application Default Credentials file."); + + File applicationDefaultCredentialsFileCopied = + Paths.get(credentialsDir.getAbsolutePath(), "application_default_credentials.json") + .toFile(); + + boolean isWindows = + System.getProperty("os.name").toLowerCase(Locale.ENGLISH).contains("windows"); + String applicationDefaultFilePathSuffix = + isWindows + ? "\\gcloud\\application_default_credentials.json" + : "/.config/gcloud/application_default_credentials.json"; + String applicationDefaultFilePath = + System.getProperty("user.home") + applicationDefaultFilePathSuffix; + + File applicationDefaultCredentialsFile = Paths.get(applicationDefaultFilePath).toFile(); + if (applicationDefaultCredentialsFile.exists()) { + Files.copy(applicationDefaultCredentialsFile, applicationDefaultCredentialsFileCopied); + } else { + throw new RuntimeException( + "Could not find the application default file: " + + applicationDefaultCredentialsFile.getAbsolutePath()); + } + } + + // Setting environment variables used by the docker-compose.yml file. + environmentVariables.put("CREDENTIALS_VOLUME", credentialsDir.getAbsolutePath()); + environmentVariables.put("TRANSFORM_SERVICE_PORT", String.valueOf(port)); + + // Building the Docker Compose command. + dockerComposeStartCommandPrefix.add("docker-compose"); + dockerComposeStartCommandPrefix.add("-p"); + dockerComposeStartCommandPrefix.add(projectName); + dockerComposeStartCommandPrefix.add("-f"); + dockerComposeStartCommandPrefix.add(dockerComposeFile.getAbsolutePath()); + } + + public void setBeamVersion(String beamVersion) { + environmentVariables.put("BEAM_VERSION", beamVersion); + } + + public void setPythonExtraPackages(String pythonExtraPackages) { + environmentVariables.put("$PYTHON_EXTRA_PACKAGES", pythonExtraPackages); + } + + public static synchronized TransformServiceLauncher forProject( + @Nullable String projectName, int port) throws IOException { + if (projectName == null || projectName.isEmpty()) { + projectName = DEFAULT_PROJECT_NAME; + } + if (!launchers.containsKey(projectName)) { + launchers.put(projectName, new TransformServiceLauncher(projectName, port)); + } + return launchers.get(projectName); + } + + private void runDockerComposeCommand(String command) throws IOException { + this.runDockerComposeCommand(command, null); + } + + private void runDockerComposeCommand(String command, @Nullable File outputOverride) + throws IOException { + List shellCommand = new ArrayList<>(); + shellCommand.addAll(dockerComposeStartCommandPrefix); + shellCommand.add(command); + System.out.println("Executing command: " + String.join(" ", command)); + ProcessBuilder processBuilder = + new ProcessBuilder(shellCommand).redirectError(ProcessBuilder.Redirect.INHERIT); + + if (outputOverride != null) { + processBuilder.redirectOutput(outputOverride); + } else { + processBuilder.redirectOutput(ProcessBuilder.Redirect.INHERIT); + } + + Map env = processBuilder.environment(); + env.putAll(this.environmentVariables); + + processBuilder.start(); + + try { + this.wait(STATUS_LOGGER_WAIT_TIME); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + public synchronized void start() throws IOException, TimeoutException { + runDockerComposeCommand("up"); + } + + public synchronized void shutdown() throws IOException { + runDockerComposeCommand("down"); + } + + public synchronized void status() throws IOException { + runDockerComposeCommand("ps"); + } + + public synchronized void waitTillUp(int timeout) throws IOException, TimeoutException { + timeout = timeout <= 0 ? DEFAULT_START_WAIT_TIME : timeout; + String statusFileName = getStatus(); + + long startTime = System.currentTimeMillis(); + while (System.currentTimeMillis() - startTime < timeout) { + try { + // We are just waiting for a local process. No need for exponential backoff. + this.wait(1000); + } catch (InterruptedException e) { + // Ignore and retry. + } + + String output = String.join(" ", java.nio.file.Files.readAllLines(Paths.get(statusFileName))); + if (!output.isEmpty()) { + if (output.contains("transform-service")) { + // Transform service was started since we found matching logs. + return; + } + } + } + + throw new TimeoutException( + "Transform Service did not start in " + timeout / 1000 + " seconds."); + } + + private synchronized String getStatus() throws IOException { + File outputOverride = File.createTempFile("output_override", null); + runDockerComposeCommand("ps", outputOverride); + + return outputOverride.getAbsolutePath(); + } + + private static class ArgConfig { + + static final String PROJECT_NAME_ARG_NAME = "project_name"; + static final String COMMAND_ARG_NAME = "command"; + static final String PORT_ARG_NAME = "port"; + static final String BEAM_VERSION_ARG_NAME = "beam_version"; + + @Option(name = "--" + PROJECT_NAME_ARG_NAME, usage = "Docker compose project name") + private String projectName = ""; + + @Option(name = "--" + COMMAND_ARG_NAME, usage = "Command to execute") + private String command = ""; + + @Option(name = "--" + PORT_ARG_NAME, usage = "Port for the transform service") + private int port = -1; + + @Option(name = "--" + BEAM_VERSION_ARG_NAME, usage = "Beam version to use.") + private String beamVersion = ""; + } + + public static void main(String[] args) throws IOException, TimeoutException { + + ArgConfig config = new ArgConfig(); + CmdLineParser parser = new CmdLineParser(config); + + try { + parser.parseArgument(args); + } catch (CmdLineException e) { + System.err.println(e.getMessage()); + System.err.println("Valid options are:"); + // print the list of available options + parser.printUsage(System.err); + System.err.println(); + + return; + } + + if (config.command.isEmpty()) { + throw new IllegalArgumentException( + "\"" + + ArgConfig.COMMAND_ARG_NAME + + "\" argument must be specified, Valid values are " + + COMMAND_POSSIBLE_VALUES); + } + if (config.beamVersion.isEmpty()) { + throw new IllegalArgumentException( + "\"" + ArgConfig.BEAM_VERSION_ARG_NAME + "\" argument must be specified."); + } + + System.out.println("==================================================="); + System.out.println( + "Starting the Beam Transform Service at " + + (config.port < 0 + ? "the default port." + : ("port " + Integer.toString(config.port) + "."))); + System.out.println("==================================================="); + + TransformServiceLauncher service = + TransformServiceLauncher.forProject(config.projectName, config.port); + if (!config.beamVersion.isEmpty()) { + service.setBeamVersion(config.beamVersion); + } + + if (config.command.equals("up")) { + service.start(); + service.waitTillUp(-1); + } else if (config.command.equals("down")) { + service.shutdown(); + } else if (config.command.equals("ps")) { + service.status(); + } else { + throw new IllegalArgumentException( + String.format("Unknown command \"%s\". Possible values are {}", config.command)); + } + } +} diff --git a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/package-info.java b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/package-info.java new file mode 100644 index 0000000000000..a1fa52e0b3b2e --- /dev/null +++ b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** A library that can be used to start up a Docker-composed based Beam transform service. */ +package org.apache.beam.sdk.transformservice.launcher; diff --git a/settings.gradle.kts b/settings.gradle.kts index cceb5a033a7b6..7a066bdaa45d1 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -250,6 +250,7 @@ include(":sdks:java:testing:test-utils") include(":sdks:java:testing:tpcds") include(":sdks:java:testing:watermarks") include(":sdks:java:transform-service") +include(":sdks:java:transform-service:launcher") include(":sdks:java:transform-service:controller-container") include(":sdks:python") include(":sdks:python:apache_beam:testing:load_tests")