[ 
https://issues.apache.org/jira/browse/BEAM-10974?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17202468#comment-17202468
 ] 

Kyle Weaver commented on BEAM-10974:
------------------------------------

Thanks Luke, you're right. I tried bisecting and found that commits prior to 
https://github.com/apache/beam/pull/12708 and  
https://github.com/apache/beam/pull/12706 also failed.

Compare the first failing build with the last passing build*:

https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/5393/testReport/org.apache.beam.sdk.transforms/
https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/5387/testReport/org.apache.beam.sdk.transforms/

We can see that before, *GroupByKeyTest was not being run at all*. I'm not sure 
why, but my bisect shows the upgrade to Gradle 6.6.1 
(https://github.com/apache/beam/pull/12776) somehow caused GroupByKeyTest to 
become included. My guess is it's related to GroupByKeyTest running with the 
junit Enclosed test runner [1].

The fact that 100MB keys are excluded [2] seems to indicate that 10MB keys must 
have worked at some time in the past. But I can't seem to run GroupByKeyTest at 
all prior to the Gradle upgrade. I get an error message:

> No tests found for given includes: [**/JvmVerification.class](exclude rules) 
> [org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testLargeKeys10MB](filter.includeTestsMatching)

So I am unable to determine yet when this regression might have actually 
occurred.

* Note that there are some builds missing in between, because these issues 
started right around the time for which Jenkins is starting to trash old jobs.

[1] 
https://github.com/apache/beam/blob/64d8c8006c4aa633438863eab15078b22b6d85ac/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java#L91
[2] 
https://github.com/apache/beam/blob/27f4836be06f38daf5741847c77e557b8b26a201/runners/flink/job-server/flink_job_server.gradle#L144

> Flink PortableValidatesRunner test failure: 
> GroupByKeyTest$BasicTests.testLargeKeys10MB
> ---------------------------------------------------------------------------------------
>
>                 Key: BEAM-10974
>                 URL: https://issues.apache.org/jira/browse/BEAM-10974
>             Project: Beam
>          Issue Type: Bug
>          Components: test-failures
>            Reporter: Robin Qiu
>            Assignee: Maximilian Michels
>            Priority: P0
>             Fix For: 2.25.0
>
>
> h3. Error Message
> java.lang.RuntimeException: The Runner experienced the following error during 
> execution: java.io.IOException: Cannot write record to fresh sort buffer. 
> Record too large.
> h3. Stacktrace
> java.lang.RuntimeException: The Runner experienced the following error during 
> execution: java.io.IOException: Cannot write record to fresh sort buffer. 
> Record too large. at 
> org.apache.beam.runners.portability.JobServicePipelineResult.propagateErrors(JobServicePipelineResult.java:165)
>  at 
> org.apache.beam.runners.portability.JobServicePipelineResult.waitUntilFinish(JobServicePipelineResult.java:110)
>  at 
> org.apache.beam.runners.portability.testing.TestPortableRunner.run(TestPortableRunner.java:83)
>  at org.apache.beam.sdk.Pipeline.run(Pipeline.java:317) at 
> org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:350) at 
> org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:331) at 
> org.apache.beam.sdk.transforms.GroupByKeyTest.runLargeKeysTest(GroupByKeyTest.java:741)
>  at 
> org.apache.beam.sdk.transforms.GroupByKeyTest.access$200(GroupByKeyTest.java:92)
>  at 
> org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testLargeKeys10MB(GroupByKeyTest.java:473)
>  at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  at java.lang.reflect.Method.invoke(Method.java:498) at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
>  at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>  at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
>  at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>  at 
> org.apache.beam.sdk.testing.TestPipeline$1.evaluate(TestPipeline.java:319) at 
> org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:266)
>  at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305) at 
> org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
>  at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:365) at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
>  at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
>  at org.junit.runners.ParentRunner$4.run(ParentRunner.java:330) at 
> org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:78) at 
> org.junit.runners.ParentRunner.runChildren(ParentRunner.java:328) at 
> org.junit.runners.ParentRunner.access$100(ParentRunner.java:65) at 
> org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:292) at 
> org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305) at 
> org.junit.runners.ParentRunner.run(ParentRunner.java:412) at 
> org.junit.runners.Suite.runChild(Suite.java:128) at 
> org.junit.runners.Suite.runChild(Suite.java:27) at 
> org.junit.runners.ParentRunner$4.run(ParentRunner.java:330) at 
> org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:78) at 
> org.junit.runners.ParentRunner.runChildren(ParentRunner.java:328) at 
> org.junit.runners.ParentRunner.access$100(ParentRunner.java:65) at 
> org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:292) at 
> org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305) at 
> org.junit.runners.ParentRunner.run(ParentRunner.java:412) at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
>  at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
>  at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
>  at 
> org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
>  at 
> org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
>  at sun.reflect.GeneratedMethodAccessor120.invoke(Unknown Source) at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  at java.lang.reflect.Method.invoke(Method.java:498) at 
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>  at 
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>  at 
> org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
>  at 
> org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
>  at com.sun.proxy.$Proxy2.processTestClass(Unknown Source) at 
> org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:119)
>  at sun.reflect.GeneratedMethodAccessor119.invoke(Unknown Source) at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  at java.lang.reflect.Method.invoke(Method.java:498) at 
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
>  at 
> org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>  at 
> org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:182)
>  at 
> org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:164)
>  at 
> org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:414)
>  at 
> org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:64)
>  at 
> org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:48)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at 
> org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:56)
>  at java.lang.Thread.run(Thread.java:748)
>  
> [https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch_PR/171/testReport/org.apache.beam.sdk.transforms/GroupByKeyTest$BasicTests/testLargeKeys10MB/]
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to