Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-19991

FileSegmentManagedBuffer performance improvement.

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 2.0.2, 2.1.0
    • 2.2.0
    • Shuffle, Spark Core
    • None

    Description

      When we do not set the value of the configuration items spark.storage.memoryMapThreshold and spark.shuffle.io.lazyFD,
      each call to the cFileSegmentManagedBuffer.nioByteBuffer or FileSegmentManagedBuffer.createInputStream method creates a NoSuchElementException instance. This is a more time-consuming operation.
      The shuffle-server thread`s stack:

      "shuffle-server-2-42" #335 daemon prio=5 os_prio=0 tid=0x00007f71e4507800 nid=0x28d12 runnable [0x00007f71af93e000]
         java.lang.Thread.State: RUNNABLE
              at java.lang.Throwable.fillInStackTrace(Native Method)
              at java.lang.Throwable.fillInStackTrace(Throwable.java:783)
              - locked <0x00000007a930f080> (a java.util.NoSuchElementException)
              at java.lang.Throwable.<init>(Throwable.java:265)
              at java.lang.Exception.<init>(Exception.java:66)
              at java.lang.RuntimeException.<init>(RuntimeException.java:62)
              at java.util.NoSuchElementException.<init>(NoSuchElementException.java:57)
              at org.apache.spark.network.yarn.util.HadoopConfigProvider.get(HadoopConfigProvider.java:38)
              at org.apache.spark.network.util.ConfigProvider.get(ConfigProvider.java:31)
              at org.apache.spark.network.util.ConfigProvider.getBoolean(ConfigProvider.java:50)
              at org.apache.spark.network.util.TransportConf.lazyFileDescriptor(TransportConf.java:157)
              at org.apache.spark.network.buffer.FileSegmentManagedBuffer.convertToNetty(FileSegmentManagedBuffer.java:132)
              at org.apache.spark.network.protocol.MessageEncoder.encode(MessageEncoder.java:54)
              at org.apache.spark.network.protocol.MessageEncoder.encode(MessageEncoder.java:33)
              at org.spark_project.io.netty.handler.codec.MessageToMessageEncoder.write(MessageToMessageEncoder.java:88)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:743)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:735)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:820)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:728)
              at org.spark_project.io.netty.handler.timeout.IdleStateHandler.write(IdleStateHandler.java:284)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:743)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:806)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:818)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:799)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:835)
              at org.spark_project.io.netty.channel.DefaultChannelPipeline.writeAndFlush(DefaultChannelPipeline.java:1017)
              at org.spark_project.io.netty.channel.AbstractChannel.writeAndFlush(AbstractChannel.java:256)
              at org.apache.spark.network.server.TransportRequestHandler.respond(TransportRequestHandler.java:194)
              at org.apache.spark.network.server.TransportRequestHandler.processFetchRequest(TransportRequestHandler.java:135)
              at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:105)
              at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:119)
              at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51)
              at org.spark_project.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:367)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:353)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:346)
              at org.spark_project.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:367)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:353)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:346)
              at org.spark_project.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:102)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:367)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:353)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:346)
              at org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:85)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:367)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:353)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:346)
              at org.spark_project.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1294)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:367)
              at org.spark_project.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:353)
              at org.spark_project.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:911)
              at org.spark_project.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
      

      Attachments

        Activity

          People

            srowen Sean R. Owen
            gq Guoqiang Li
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: