You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Sean Zhong (JIRA)" <ji...@apache.org> on 2016/09/01 06:35:20 UTC

[jira] [Commented] (SPARK-17356) Out of memory when calling TreeNode.toJSON

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

Sean Zhong commented on SPARK-17356:
------------------------------------

*Analysis*

After looking at the mmap, there is a suspicious line
{code}
  20:             1        8388624  [Lorg.apache.spark.ml.attribute.Attribute;
{code}

This means a single Attribute array takes more than 8388624 bytes, and if each reference takes 8 bytes, it means there are 1 million attributes.

The array probably is used in AttributeGroup, whose signature is:
{code}
class AttributeGroup private (
    val name: String,
    val numAttributes: Option[Int],
    attrs: Option[Array[Attribute]])
{code}

And, in AttributeGroup, there is a toMetaData function which will convert the Attribute array to Meta data
{code}
  def toMetadata(): Metadata = toMetadata(Metadata.empty)
{code}

Finally, the metadata are saved to expression Attribute.
 
For example, in org.apache.spark.ml.feature.Interaction transform function, the meta data is set to attribute of Alias expression, when aliasing the udf function like this:

{code}
  override def transform(dataset: Dataset[_]): DataFrame = {
    ...
    // !NOTE!: This is an attribute group
    val featureAttrs = getFeatureAttrs(inputFeatures)

    def interactFunc = udf { row: Row =>
      ...
    }

    val featureCols = inputFeatures.map { f =>
      f.dataType match {
        case DoubleType => dataset(f.name)
        case _: VectorUDT => dataset(f.name)
        case _: NumericType | BooleanType => dataset(f.name).cast(DoubleType)
      }
    }

    // !NOTE!: The meta data i stored in Alias expresion by function call .as(..., featureAttrs.toMetadata())
    dataset.select(
      col("*"),
      interactFunc(struct(featureCols: _*)).as($(outputCol), featureAttrs.toMetadata()))
  }

{code}

And, when calling toJSON, the metaData will be converted to JSON.

> Out of memory when calling TreeNode.toJSON
> ------------------------------------------
>
>                 Key: SPARK-17356
>                 URL: https://issues.apache.org/jira/browse/SPARK-17356
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>            Reporter: Sean Zhong
>         Attachments: jmap.txt, jstack.txt, queryplan.txt
>
>
> When using MLLib, when calling toJSON on a plan with many level of sub-queries, it may cause out of memory exception with stack trace like this
> {code}
> java.lang.OutOfMemoryError: GC overhead limit exceeded
> 	at scala.collection.mutable.AbstractSeq.<init>(Seq.scala:47)
> 	at scala.collection.mutable.AbstractBuffer.<init>(Buffer.scala:48)
> 	at scala.collection.mutable.ListBuffer.<init>(ListBuffer.scala:46)
> 	at scala.collection.immutable.List$.newBuilder(List.scala:396)
> 	at scala.collection.generic.GenericTraversableTemplate$class.newBuilder(GenericTraversableTemplate.scala:64)
> 	at scala.collection.AbstractTraversable.newBuilder(Traversable.scala:105)
> 	at scala.collection.TraversableLike$class.filter(TraversableLike.scala:262)
> 	at scala.collection.AbstractTraversable.filter(Traversable.scala:105)
> 	at scala.collection.TraversableLike$class.filterNot(TraversableLike.scala:274)
> 	at scala.collection.AbstractTraversable.filterNot(Traversable.scala:105)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:25)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:20)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:25)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:25)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:25)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:25)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:20)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:20)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:25)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:20)
> 	at org.json4s.jackson.JValueSerializer.serialize(JValueSerializer.scala:7)
> 	at com.fasterxml.jackson.databind.ser.DefaultSerializerProvider.serializeValue(DefaultSerializerProvider.java:128)
> 	at com.fasterxml.jackson.databind.ObjectMapper._configAndWriteValue(ObjectMapper.java:2881)
> 	at com.fasterxml.jackson.databind.ObjectMapper.writeValueAsString(ObjectMapper.java:2338)
> 	at org.json4s.jackson.JsonMethods$class.compact(JsonMethods.scala:34)
> 	at org.json4s.jackson.JsonMethods$.compact(JsonMethods.scala:50)
> 	at org.apache.spark.sql.catalyst.trees.TreeNode.toJSON(TreeNode.scala:566)
> {code}
> The query plan, stack trace, and jmap distribution is attached.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org