lostluck commented on code in PR #32294:
URL: https://github.com/apache/beam/pull/32294#discussion_r1728068353
##########
runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismRunner.java:
##########
@@ -34,34 +40,38 @@
* submit to an already running Prism service, use the {@link PortableRunner}
with the {@link
* PortablePipelineOptions#getJobEndpoint()} option instead. Prism is a {@link
* org.apache.beam.runners.portability.PortableRunner} maintained at <a
- *
href="https://github.com/apache/beam/tree/master/sdks/go/cmd/prism">sdks/go/cmd/prism</a>.
+ *
href="https://github.com/apache/beam/tree/master/sdks/go/cmd/prism">sdks/go/cmd/prism</a>.
For
+ * testing, use {@link TestPrismRunner}.
*/
-// TODO(https://github.com/apache/beam/issues/31793): add public modifier
after finalizing
-// PrismRunner. Depends on: https://github.com/apache/beam/issues/31402 and
-// https://github.com/apache/beam/issues/31792.
-class PrismRunner extends PipelineRunner<PipelineResult> {
+public class PrismRunner extends PipelineRunner<PipelineResult> {
private static final Logger LOG = LoggerFactory.getLogger(PrismRunner.class);
- private static final String DEFAULT_PRISM_ENDPOINT = "localhost:8073";
-
- private final PortableRunner internal;
private final PrismPipelineOptions prismPipelineOptions;
- private PrismRunner(PortableRunner internal, PrismPipelineOptions
prismPipelineOptions) {
- this.internal = internal;
+ protected PrismRunner(PrismPipelineOptions prismPipelineOptions) {
this.prismPipelineOptions = prismPipelineOptions;
}
+ PrismPipelineOptions getPrismPipelineOptions() {
+ return prismPipelineOptions;
+ }
+
/**
* Invoked from {@link Pipeline#run} where {@link PrismRunner} instantiates
using {@link
* PrismPipelineOptions} configuration details.
*/
public static PrismRunner fromOptions(PipelineOptions options) {
PrismPipelineOptions prismPipelineOptions =
options.as(PrismPipelineOptions.class);
+ validate(prismPipelineOptions);
assignDefaultsIfNeeded(prismPipelineOptions);
- PortableRunner internal = PortableRunner.fromOptions(options);
- return new PrismRunner(internal, prismPipelineOptions);
+ return new PrismRunner(prismPipelineOptions);
+ }
+
+ private static void validate(PrismPipelineOptions options) {
+ checkArgument(
+ Strings.isNullOrEmpty(options.getJobEndpoint()),
+ "when specifying --jobEndpoint, use --runner=PortableRunner instead");
Review Comment:
Excellent. Good.
##########
runners/prism/java/src/main/java/org/apache/beam/runners/prism/TestPrismRunner.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.runners.prism;
+
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.function.Supplier;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+
+/**
+ * {@link TestPrismRunner} is the recommended {@link PipelineRunner} to use
for tests that rely on
+ * <a
href="https://github.com/apache/beam/tree/master/sdks/go/cmd/prism">sdks/go/cmd/prism</a>.
See
+ * {@link PrismRunner} for more details.
+ */
+public class TestPrismRunner extends PipelineRunner<PipelineResult> {
+
+ private final PrismRunner internal;
+ private final TestPrismPipelineOptions prismPipelineOptions;
+
+ /**
+ * Invoked from {@link Pipeline#run} where {@link TestPrismRunner}
instantiates using {@link
+ * TestPrismPipelineOptions} configuration details.
+ */
+ public static TestPrismRunner fromOptions(PipelineOptions options) {
+ TestPrismPipelineOptions prismPipelineOptions =
options.as(TestPrismPipelineOptions.class);
+ PrismRunner delegate = PrismRunner.fromOptions(options);
+ return new TestPrismRunner(delegate, prismPipelineOptions);
+ }
+
+ private TestPrismRunner(PrismRunner internal, TestPrismPipelineOptions
options) {
+ this.internal = internal;
+ this.prismPipelineOptions = options;
+ }
+
+ TestPrismPipelineOptions getTestPrismPipelineOptions() {
+ return prismPipelineOptions;
+ }
+
+ @Override
+ public PipelineResult run(Pipeline pipeline) {
+ PipelineResult result = internal.run(pipeline);
+ PipelineResult.State state = getWaitUntilFinishRunnable(result).get();
+ assertThat(
Review Comment:
I know this is supposed to be in test code, but it feels weird to have/use a
hamcrest assert in production instead of the AssertionError that other things
expect?
We can resolve that later as we become more comfortable with the Java test
stack.
##########
runners/prism/java/src/main/java/org/apache/beam/runners/prism/TestPrismRunner.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.runners.prism;
+
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.function.Supplier;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+
+/**
+ * {@link TestPrismRunner} is the recommended {@link PipelineRunner} to use
for tests that rely on
+ * <a
href="https://github.com/apache/beam/tree/master/sdks/go/cmd/prism">sdks/go/cmd/prism</a>.
See
+ * {@link PrismRunner} for more details.
+ */
+public class TestPrismRunner extends PipelineRunner<PipelineResult> {
+
+ private final PrismRunner internal;
+ private final TestPrismPipelineOptions prismPipelineOptions;
+
+ /**
+ * Invoked from {@link Pipeline#run} where {@link TestPrismRunner}
instantiates using {@link
+ * TestPrismPipelineOptions} configuration details.
+ */
+ public static TestPrismRunner fromOptions(PipelineOptions options) {
+ TestPrismPipelineOptions prismPipelineOptions =
options.as(TestPrismPipelineOptions.class);
+ PrismRunner delegate = PrismRunner.fromOptions(options);
+ return new TestPrismRunner(delegate, prismPipelineOptions);
+ }
+
+ private TestPrismRunner(PrismRunner internal, TestPrismPipelineOptions
options) {
+ this.internal = internal;
+ this.prismPipelineOptions = options;
+ }
+
+ TestPrismPipelineOptions getTestPrismPipelineOptions() {
+ return prismPipelineOptions;
+ }
+
+ @Override
+ public PipelineResult run(Pipeline pipeline) {
+ PipelineResult result = internal.run(pipeline);
+ PipelineResult.State state = getWaitUntilFinishRunnable(result).get();
+ assertThat(
Review Comment:
Note, this is what TestDataflowRunner and the TestPortableRunner do, so it's
fine for now.
##########
runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismRunner.java:
##########
@@ -72,15 +82,44 @@ public PipelineResult run(Pipeline pipeline) {
prismPipelineOptions.getDefaultEnvironmentType(),
prismPipelineOptions.getJobEndpoint());
- return internal.run(pipeline);
+ try {
+ PrismExecutor executor = startPrism();
+ PortableRunner delegate =
PortableRunner.fromOptions(prismPipelineOptions);
+ return new PrismPipelineResult(delegate.run(pipeline), executor::stop);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ PrismExecutor startPrism() throws IOException {
+ PrismLocator locator = new PrismLocator(prismPipelineOptions);
+ int port = findAvailablePort();
+ String portFlag = String.format(PrismExecutor.JOB_PORT_FLAG_TEMPLATE,
port);
+ String serveHttpFlag =
+ String.format(
+ PrismExecutor.SERVE_HTTP_FLAG_TEMPLATE,
prismPipelineOptions.getEnableWebUI());
Review Comment:
Just need to add the --
idle_shutdown_timeout=5m flag
(That is, shutdown the prism instance if it's been idle for 5 minutes) to
avoid dangling processes. We can make this shorter too TBH, but lets start
conservative.
##########
runners/prism/java/src/main/java/org/apache/beam/runners/prism/TestPrismRunner.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.runners.prism;
+
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.function.Supplier;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+
+/**
+ * {@link TestPrismRunner} is the recommended {@link PipelineRunner} to use
for tests that rely on
+ * <a
href="https://github.com/apache/beam/tree/master/sdks/go/cmd/prism">sdks/go/cmd/prism</a>.
See
+ * {@link PrismRunner} for more details.
+ */
+public class TestPrismRunner extends PipelineRunner<PipelineResult> {
+
+ private final PrismRunner internal;
+ private final TestPrismPipelineOptions prismPipelineOptions;
+
+ /**
+ * Invoked from {@link Pipeline#run} where {@link TestPrismRunner}
instantiates using {@link
+ * TestPrismPipelineOptions} configuration details.
+ */
+ public static TestPrismRunner fromOptions(PipelineOptions options) {
+ TestPrismPipelineOptions prismPipelineOptions =
options.as(TestPrismPipelineOptions.class);
+ PrismRunner delegate = PrismRunner.fromOptions(options);
+ return new TestPrismRunner(delegate, prismPipelineOptions);
+ }
+
+ private TestPrismRunner(PrismRunner internal, TestPrismPipelineOptions
options) {
+ this.internal = internal;
+ this.prismPipelineOptions = options;
+ }
+
+ TestPrismPipelineOptions getTestPrismPipelineOptions() {
+ return prismPipelineOptions;
+ }
+
+ @Override
+ public PipelineResult run(Pipeline pipeline) {
+ PipelineResult result = internal.run(pipeline);
+ PipelineResult.State state = getWaitUntilFinishRunnable(result).get();
+ assertThat(
+ "Pipeline did not succeed. Check Prism logs for further details.",
+ state,
+ Matchers.is(PipelineResult.State.DONE));
+ return result;
+ }
+
+ private Supplier<PipelineResult.State>
getWaitUntilFinishRunnable(PipelineResult result) {
+ if (prismPipelineOptions.getTestTimeoutSeconds() != null) {
+ Long testTimeoutSeconds =
checkStateNotNull(prismPipelineOptions.getTestTimeoutSeconds());
+ return () ->
result.waitUntilFinish(Duration.standardSeconds(testTimeoutSeconds));
Review Comment:
OOO clever. SDK side test waiting. I like it.
--
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]