Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

The PostCommit Java PVR Spark Batch job is flaky #30512

Open
github-actions bot opened this issue Mar 5, 2024 · 6 comments · Fixed by #32723
Open

The PostCommit Java PVR Spark Batch job is flaky #30512

github-actions bot opened this issue Mar 5, 2024 · 6 comments · Fixed by #32723

Comments

@github-actions
Copy link
Contributor

github-actions bot commented Mar 5, 2024

The PostCommit Java PVR Spark Batch is failing over 50% of the time
Please visit https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml?query=is%3Afailure+branch%3Amaster to see the logs.

@Abacn
Copy link
Contributor

Abacn commented May 14, 2024

Random tests in this test suite failing due to tmp file get deleted half way, likely a racing issue.

This is recurring for a long time.

java.lang.RuntimeException: The Runner experienced the following error during execution:
java.lang.RuntimeException: Error received from SDK harness for instruction 4: java.lang.IllegalArgumentException: unable to deserialize Custom DoFn With Execution Info
	at org.apache.beam.sdk.util.SerializableUtils.deserializeFromByteArray(SerializableUtils.java:78)
	at org.apache.beam.sdk.util.construction.ParDoTranslation.doFnWithExecutionInformationFromProto(ParDoTranslation.java:767)
	at org.apache.beam.sdk.util.construction.ParDoTranslation.getDoFn(ParDoTranslation.java:391)
	at org.apache.beam.fn.harness.FnApiDoFnRunner.<init>(FnApiDoFnRunner.java:379)
	...
	at java.lang.Thread.run(Thread.java:750)
Caused by: java.io.InvalidClassException: org.apache.beam.sdk.transforms.ParDoTest$StateTests; unable to create instance
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2200)
	...
	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:461)
	at org.apache.beam.sdk.util.SerializableUtils.deserializeFromByteArray(SerializableUtils.java:75)
	... 19 more
Caused by: java.lang.reflect.InvocationTargetException
	at sun.reflect.GeneratedSerializationConstructorAccessor390.newInstance(Unknown Source)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at java.io.ObjectStreamClass.newInstance(ObjectStreamClass.java:1102)
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2197)
	... 31 more
Caused by: java.util.ServiceConfigurationError: org.apache.beam.sdk.io.FileSystemRegistrar: Error reading configuration file
	...
	at org.apache.beam.sdk.transforms.ParDoTest$SharedTestBase.<init>(ParDoTest.java:193)
	... 35 more
Caused by: java.io.FileNotFoundException: /tmp/spark-4426f648-ee8b-49e3-9545-dffde22d1a6f/userFiles-b02ee903-820b-4b77-b1b7-69f020d54c8b/beam-sdks-java-extensions-google-cloud-platform-core-2.57.0-SNAPSHOT.jar (No such file or directory)
	at java.util.zip.ZipFile.open(Native Method)
	...
	at java.net.URL.openStream(URL.java:1093)
	at java.util.ServiceLoader.parse(ServiceLoader.java:304)
	... 46 more

	at org.apache.beam.runners.portability.JobServicePipelineResult.propagateErrors(JobServicePipelineResult.java:176)
	at org.apache.beam.runners.portability.JobServicePipelineResult.waitUntilFinish(JobServicePipelineResult.java:117)
	at org.apache.beam.runners.portability.testing.TestPortableRunner.run(TestPortableRunner.java:81)
	at org.apache.beam.sdk.Pipeline.run(Pipeline.java:325)
	at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:404)
	...
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.lang.Thread.run(Thread.java:750)

Copy link
Contributor Author

Reopening since the workflow is still flaky

@Abacn
Copy link
Contributor

Abacn commented Oct 9, 2024

Taking a closer look, tests are failing on initializing TestPipeline:

Caused by: java.io.InvalidClassException: org.apache.beam.sdk.transforms.ParDoTest$LifecycleTests; unable to create instance
Caused by: java.lang.reflect.InvocationTargetException
Caused by: java.util.ServiceConfigurationError: org.apache.beam.sdk.io.FileSystemRegistrar: Error reading configuration file
...
at org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists.newArrayList(Lists.java:132)
	at org.apache.beam.sdk.io.FileSystems.setDefaultPipelineOptions(FileSystems.java:587)
	at org.apache.beam.sdk.testing.TestPipeline.testingPipelineOptions(TestPipeline.java:518)
	at org.apache.beam.sdk.testing.TestPipeline.create(TestPipeline.java:273)
...
Caused by: java.io.FileNotFoundException: /tmp/spark-f76d7a05-d782-41a9-9926-089251e24bdd/userFiles-7506344e-2e1b-43ea-8ccb-5bbc936d955f/beam-runners-direct-java-2.60.0-SNAPSHOT.jar (No such file or directory)
	at java.util.zip.ZipFile.open(Native Method)
at java.net.URL.openStream(URL.java:1093)
	at java.util.ServiceLoader.parse(ServiceLoader.java:304)

the failing call is "setDefaultPipelineOptions", which loads filesystem registrar that are autoService. It sounds similar to google/auto#718

Another observation is that the test trying to read from /tmp/spark-*** which looks like the staged jars for Spark runner pipeline. However the invocation is inside junit code, and it looks like the staged jar paths leaked into the host test.

@Abacn
Copy link
Contributor

Abacn commented Oct 9, 2024

Run tests locally (macOS). There are lots of this log:

[spark-runner-job-invoker-0] INFO org.apache.spark.executor.Executor - Adding file:/private/var/folders/wg/hwmcqjwd4zz75mjs0r5z_3f400y2yj/T/spark-fad6dd06-5673-4920-9176-3a801211fa9d/userFiles-f57ec6cf-273d-46c9-b251-ff365803ad6a/commons-lang-2.6.jar to class loader

and there is also intermittent test failure, but now FileNotFoundException has a hint (Too many files open)

Copy link
Contributor Author

Reopening since the workflow is still flaky

@Abacn
Copy link
Contributor

Abacn commented Oct 28, 2024

Likely due to worker upgraded to Java 11, however the actual error is not surfaced. A first step would be making the "FnHarness Startup failed" report the underlying Exception, making it debuggable.

This test has been a trouble maker as Spark in memory runner doing dirty ops on class loader and often caused conflict. Remove milestone for now until we have bandwidth to look into it

UPDATE: added log to serface the error: 76a320e

It reveals the root cause:

java.util.ServiceConfigurationError: org.apache.beam.sdk.harness.JvmInitializer: Error accessing configuration file
java.util.concurrent.ExecutionException: java.util.ServiceConfigurationError: org.apache.beam.sdk.harness.JvmInitializer: Error accessing configuration file
	at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
	at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
	at org.apache.beam.runners.fnexecution.environment.EmbeddedEnvironmentFactory.lambda$createEnvironment$1(EmbeddedEnvironmentFactory.java:141)
	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
	at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: java.util.ServiceConfigurationError: org.apache.beam.sdk.harness.JvmInitializer: Error accessing configuration file
	at java.base/java.util.ServiceLoader.fail(ServiceLoader.java:582)
	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.parse(ServiceLoader.java:1173)
	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.nextProviderClass(ServiceLoader.java:1206)
	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNextService(ServiceLoader.java:1221)
	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.hasNext(ServiceLoader.java:1265)
	at java.base/java.util.ServiceLoader$2.hasNext(ServiceLoader.java:1300)
	at java.base/java.util.ServiceLoader$3.hasNext(ServiceLoader.java:1385)
	at org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection$Builder.addAll(ImmutableCollection.java:474)
	at org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet$Builder.addAll(ImmutableSet.java:549)
	at org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSortedSet$Builder.addAll(ImmutableSortedSet.java:534)
	at org.apache.beam.sdk.util.common.ReflectHelpers.loadServicesOrdered(ReflectHelpers.java:200)
	at org.apache.beam.sdk.util.common.ReflectHelpers.loadServicesOrdered(ReflectHelpers.java:212)
	at org.apache.beam.sdk.fn.JvmInitializers.runBeforeProcessing(JvmInitializers.java:53)
	at org.apache.beam.fn.harness.FnHarness.main(FnHarness.java:400)
	at org.apache.beam.runners.fnexecution.environment.EmbeddedEnvironmentFactory.lambda$createEnvironment$0(EmbeddedEnvironmentFactory.java:110)
	... 4 more
Caused by: java.nio.file.NoSuchFileException: /private/var/folders/wg/hwmcqjwd4zz75mjs0r5z_3f400y2yj/T/spark-c1f96f55-2119-4459-aca5-9e0e28408b7f/userFiles-936c0851-8610-41fe-a16a-c4b64f157cb0/beam-sdks-java-core-2.61.0-SNAPSHOT-tests.jar
	at java.base/sun.nio.fs.UnixException.translateToIOException(UnixException.java:92)
	at java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:111)
	at java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:116)
	at java.base/sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55)
	at java.base/sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:149)
	at java.base/java.nio.file.Files.readAttributes(Files.java:1764)
	at java.base/java.util.zip.ZipFile$Source.get(ZipFile.java:1259)
	at java.base/java.util.zip.ZipFile$CleanableResource.<init>(ZipFile.java:831)
	at java.base/java.util.zip.ZipFile$CleanableResource$FinalizableResource.<init>(ZipFile.java:857)
	at java.base/java.util.zip.ZipFile$CleanableResource.get(ZipFile.java:846)
	at java.base/java.util.zip.ZipFile.<init>(ZipFile.java:248)
	at java.base/java.util.zip.ZipFile.<init>(ZipFile.java:177)
	at java.base/java.util.jar.JarFile.<init>(JarFile.java:350)
	at java.base/sun.net.www.protocol.jar.URLJarFile.<init>(URLJarFile.java:103)
	at java.base/sun.net.www.protocol.jar.URLJarFile.getJarFile(URLJarFile.java:72)
	at java.base/sun.net.www.protocol.jar.JarFileFactory.get(JarFileFactory.java:99)
	at java.base/sun.net.www.protocol.jar.JarURLConnection.connect(JarURLConnection.java:125)
	at java.base/sun.net.www.protocol.jar.JarURLConnection.getInputStream(JarURLConnection.java:155)
	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.parse(ServiceLoader.java:1165)
	... 17 more

This is very similar to #30512 (comment), the same ClassLoader issue now happened in org.apache.beam.sdk.util.common.ReflectHelpers.loadServicesOrdered when FnHarness is started in memory

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants