You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Alexey Romanenko (Jira)" <ji...@apache.org> on 2022/03/14 18:05:00 UTC

[jira] [Created] (BEAM-14099) Pipeline with large number of PTransforms fails with StackOverflowError

Alexey Romanenko created BEAM-14099:
---------------------------------------

             Summary: Pipeline with large number of PTransforms fails with StackOverflowError 
                 Key: BEAM-14099
                 URL: https://issues.apache.org/jira/browse/BEAM-14099
             Project: Beam
          Issue Type: Bug
          Components: runner-spark, sdk-java-core
            Reporter: Alexey Romanenko


If pipeline, written in Java SDK, contains a large number of PTransforms then it fails with a  {{java.lang.StackOverflowError}}

Code snippet to reproduce (based on WordCount example):
{code}
public class WordCountWithNFilters {
  private static final int N = 100;

  public static void main(String[] args) {
    PipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().create();
    Pipeline p = Pipeline.create(options);

    PCollection<String> words = p.apply(TextIO.read().from("file://tmp/input.txt"))
        .apply(
            FlatMapElements.into(TypeDescriptors.strings())
                .via((String line) -> Arrays.asList(line.split("[^\\p{L}]+"))));
    for (int i = 0; i < N; i++) {
      words = words.apply(Filter.by((String word) -> !word.isEmpty()));
    }
    words.apply(Count.perElement())
        .apply(
            MapElements.into(TypeDescriptors.strings())
                .via(
                    (KV<String, Long> wordCount) ->
                        wordCount.getKey() + ": " + wordCount.getValue()))
        .apply(TextIO.write().to("wordcounts"));

    p.run().waitUntilFinish();
  }
}
{code}

Log while running with SparkRunner:
{code}
2022-03-14 19:01:30,465 [pool-3-thread-1] INFO  org.apache.beam.runners.spark.SparkRunner$Evaluator  - Evaluating View.CreatePCollectionView
[WARNING] 
org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.StackOverflowError
    at org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom (SparkPipelineResult.java:73)
    at org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish (SparkPipelineResult.java:104)
    at org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish (SparkPipelineResult.java:92)
    at org.apache.beam.samples.sql.WordCountWithNFilters.main (WordCountWithNFilters.java:39)
    at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:254)
    at java.lang.Thread.run (Thread.java:748)
Caused by: java.lang.StackOverflowError
    at java.lang.ReflectiveOperationException.<init> (ReflectiveOperationException.java:89)
    at java.lang.reflect.InvocationTargetException.<init> (InvocationTargetException.java:72)
    at sun.reflect.GeneratedMethodAccessor39.invoke (Unknown Source)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke (Method.java:498)
    at java.io.ObjectStreamClass.invokeWriteReplace (ObjectStreamClass.java:1244)
    at java.io.ObjectOutputStream.writeObject0 (ObjectOutputStream.java:1136)
    at java.io.ObjectOutputStream.defaultWriteFields (ObjectOutputStream.java:1548)
    at java.io.ObjectOutputStream.writeSerialData (ObjectOutputStream.java:1509)
    at java.io.ObjectOutputStream.writeOrdinaryObject (ObjectOutputStream.java:1432)
    at java.io.ObjectOutputStream.writeObject0 (ObjectOutputStream.java:1178)
    at java.io.ObjectOutputStream.defaultWriteFields (ObjectOutputStream.java:1548)
    at java.io.ObjectOutputStream.writeSerialData (ObjectOutputStream.java:1509)
    at java.io.ObjectOutputStream.writeOrdinaryObject (ObjectOutputStream.java:1432)
    at java.io.ObjectOutputStream.writeObject0 (ObjectOutputStream.java:1178)
    at java.io.ObjectOutputStream.writeObject (ObjectOutputStream.java:348)
    at scala.collection.immutable.List$SerializationProxy.writeObject (List.scala:479)
    at sun.reflect.GeneratedMethodAccessor40.invoke (Unknown Source)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43)
...
{code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)