[
https://issues.apache.org/jira/browse/FLINK-26979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17555368#comment-17555368
]
Shubham Bansal commented on FLINK-26979:
----------------------------------------
I tried to run the JsonFunctionsITCase.test many times but I was not able to
reproduce it. Then I looked at the pipeline run link from [~hxbks2ks] which has
a similar stack trace but for a different test.
Then looking at [~slinkydeveloper] 's comment about parallel execution and the
transformation code according to the stack trace we have in both the failure.
Here is what I found:
{code:java}
// This is used to assign a unique ID to every Transformation
protected static Integer idCounter = 0;
public static int getNewNodeId() {
idCounter++;
return idCounter;
}
{code}
This code is in
{noformat}
src/main/java/org/apache/flink/api/dag/Transformation.java{noformat}
which, if called from multiple threads to initialize transformation, can cause
a race condition on the *idCounter* variable. This is why the same id is being
created for different transformations when run in parallel.
[~martijnvisser] Does that sound like a reasonable RCA? Or I should try to
reproduce the issue?
If yes, then one way of fixing it is by making the *getNewNodeId* function
{_}synchronized{_}. Another way is to make the *idCounter* as
{_}AtomicInteger{_}.
Let me know what you think. I am still familiarizing myself with the coding
standard of Apache Flink so I have given options to fix it. Not sure which one
is suitable according to the coding standard, although I see the usage of both
synchronized and AtomicInteger in the codebase.
Thanks,
Shubham
> JsonFunctionsITCase.test failed on azure
> ----------------------------------------
>
> Key: FLINK-26979
> URL: https://issues.apache.org/jira/browse/FLINK-26979
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Planner
> Affects Versions: 1.16.0
> Reporter: Yun Gao
> Assignee: Shubham Bansal
> Priority: Critical
> Labels: test-stability
>
> {code:java}
> Mar 31 04:38:37 [ERROR] Tests run: 140, Failures: 0, Errors: 1, Skipped: 0,
> Time elapsed: 11.301 s <<< FAILURE! - in
> org.apache.flink.table.planner.functions.JsonFunctionsITCase
> Mar 31 04:38:37 [ERROR]
> org.apache.flink.table.planner.functions.JsonFunctionsITCase.test(TestCase)[64]
> Time elapsed: 0.738 s <<< ERROR!
> Mar 31 04:38:37 java.lang.RuntimeException: Duplicate vertexID 452
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraph.addNode(StreamGraph.java:504)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraph.addOperator(StreamGraph.java:403)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraph.addOperator(StreamGraph.java:382)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.runtime.translators.AbstractOneInputTransformationTranslator.translateInternal(AbstractOneInputTransformationTranslator.java:63)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.runtime.translators.OneInputTransformationTranslator.translateForStreamingInternal(OneInputTransformationTranslator.java:65)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.runtime.translators.OneInputTransformationTranslator.translateForStreamingInternal(OneInputTransformationTranslator.java:37)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.SimpleTransformationTranslator.translateForStreaming(SimpleTransformationTranslator.java:62)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraphGenerator.translate(StreamGraphGenerator.java:825)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraphGenerator.transform(StreamGraphGenerator.java:555)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraphGenerator.getParentInputIds(StreamGraphGenerator.java:846)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraphGenerator.translate(StreamGraphGenerator.java:804)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraphGenerator.transform(StreamGraphGenerator.java:555)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.graph.StreamGraphGenerator.generate(StreamGraphGenerator.java:316)
> Mar 31 04:38:37 at
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.generateStreamGraph(StreamExecutionEnvironment.java:2153)
> Mar 31 04:38:37 at
> org.apache.flink.table.planner.delegation.DefaultExecutor.createPipeline(DefaultExecutor.java:83)
> Mar 31 04:38:37 at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeQueryOperation(TableEnvironmentImpl.java:832)
> Mar 31 04:38:37 at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1317)
> Mar 31 04:38:37 at
> org.apache.flink.table.api.internal.TableImpl.execute(TableImpl.java:475)
> Mar 31 04:38:37 at
> org.apache.flink.table.planner.functions.BuiltInFunctionTestBase$ResultTestItem.test(BuiltInFunctionTestBase.java:354)
> Mar 31 04:38:37 at
> org.apache.flink.table.planner.functions.BuiltInFunctionTestBase$TestSetSpec.lambda$getTestCase$4(BuiltInFunctionTestBase.java:320)
> Mar 31 04:38:37 at
> org.apache.flink.table.planner.functions.BuiltInFunctionTestBase$TestCase.execute(BuiltInFunctionTestBase.java:113)
> Mar 31 04:38:37 at
> org.apache.flink.table.planner.functions.BuiltInFunctionTestBase.test(BuiltInFunctionTestBase.java:93)
> Mar 31 04:38:37 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native
> Method)
> Mar 31 04:38:37 at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> Mar 31 04:38:37 at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> Mar 31 04:38:37 at java.lang.reflect.Method.invoke(Method.java:498)
> Mar 31 04:38:37 at
> org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:725)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.execution.InvocationInterceptorChain$ValidatingInvocation.proceed(InvocationInterceptorChain.java:131)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.extension.TimeoutExtension.intercept(TimeoutExtension.java:149)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestableMethod(TimeoutExtension.java:140)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestTemplateMethod(TimeoutExtension.java:92)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.execution.ExecutableInvoker$ReflectiveInterceptorCall.lambda$ofVoidMethod$0(ExecutableInvoker.java:115)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.execution.ExecutableInvoker.lambda$invoke$0(ExecutableInvoker.java:105)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.execution.InvocationInterceptorChain$InterceptedInvocation.proceed(InvocationInterceptorChain.java:106)
> Mar 31 04:38:37 at
> org.junit.jupiter.engine.execution.InvocationInterceptorChain.proceed(InvocationInterceptorChain.java:64)
> {code}
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33999&view=logs&j=f2c100be-250b-5e85-7bbe-176f68fcddc5&t=05efd11e-5400-54a4-0d27-a4663be008a9&l=10648
--
This message was sent by Atlassian Jira
(v8.20.7#820007)