XMLWordPrintableJSON

Details

    Description

      A number of tests fails like e.g.for
      org.apache.flink.table.runtime.operators.python.scalar.arrow.ArrowPythonScalarFunctionOpe

      [ERROR] ratorTest.testFinishBundleTriggeredByTime  Time elapsed: 0.031 s  <<< ERROR!
      java.lang.NoClassDefFoundError: Could not initialize class org.apache.flink.table.runtime.arrow.serializers.ArrowSerializer
      	at org.apache.flink.table.runtime.operators.python.scalar.arrow.ArrowPythonScalarFunctionOperator.open(ArrowPythonScalarFunctionOperator.java:72)
      	at org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness.open(AbstractStreamOperatorTestHarness.java:681)
      	at org.apache.flink.table.runtime.operators.python.scalar.PythonScalarFunctionOperatorTestBase.testFinishBundleTriggeredByTime(PythonScalarFunctionOperatorTestBase.java:156)
      	at java.base/jdk.internal.reflect.DirectMethodHandleAccessor.invoke(DirectMethodHandleAccessor.java:103)
      ...
      Caused by: java.lang.ExceptionInInitializerError: Exception java.lang.RuntimeException: Arrow depends on DirectByteBuffer.<init>(long, int) which is not available. Please set the system property 'io.netty.tryReflectionSetAccessible' to 'true'. [in thread "ForkJoinPool-3-worker-1"]
          at org.apache.flink.table.runtime.arrow.ArrowUtils.checkArrowUsable(ArrowUtils.java:184)
          at org.apache.flink.table.runtime.arrow.serializers.ArrowSerializer.<clinit>(ArrowSerializer.java:44)
          at org.apache.flink.table.runtime.utils.PassThroughPythonAggregateFunctionRunner.<init>(PassThroughPythonAggregateFunctionRunner.java:96)
          at org.apache.flink.table.runtime.operators.python.aggregate.arrow.batch.BatchArrowPythonGroupWindowAggregateFunctionOperatorTest$PassThroughBatchArrowPythonGroupWindowAggregateFunctionOperator.createPythonFunctionRunner(BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java:414)
          at org.apache.flink.streaming.api.operators.python.process.AbstractExternalPythonFunctionOperator.open(AbstractExternalPythonFunctionOperator.java:56)
          at org.apache.flink.table.runtime.operators.python.AbstractStatelessFunctionOperator.open(AbstractStatelessFunctionOperator.java:92)
          at org.apache.flink.table.runtime.operators.python.aggregate.arrow.AbstractArrowPythonAggregateFunctionOperator.open(AbstractArrowPythonAggregateFunctionOperator.java:89)
          at org.apache.flink.table.runtime.operators.python.aggregate.arrow.batch.AbstractBatchArrowPythonAggregateFunctionOperator.open(AbstractBatchArrowPythonAggregateFunctionOperator.java:82)
          at org.apache.flink.table.runtime.operators.python.aggregate.arrow.batch.BatchArrowPythonGroupWindowAggregateFunctionOperator.open(BatchArrowPythonGroupWindowAggregateFunctionOperator.java:119)
          at org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness.open(AbstractStreamOperatorTestHarness.java:681)
          at org.apache.flink.table.runtime.operators.python.aggregate.arrow.batch.BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.testFinishBundleTriggeredByCount(BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java:140)
          ... 57 more
      

      UPDATE
      The reason is that since JDK21 there was removed this constructor within https://bugs.openjdk.org/browse/JDK-8303083
       and corresponding changes in Netty are done at https://github.com/netty/netty/pull/13366 which is a part of 4.1.93.Final

      Attachments

        Issue Links

          Activity

            People

              martijnvisser Martijn Visser
              Sergey Nuyanzin Sergey Nuyanzin
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: