You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "xufusheng (JIRA)" <ji...@apache.org> on 2017/05/09 08:29:04 UTC

[jira] [Comment Edited] (ARROW-855) Arrow Memory Leak

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

xufusheng edited comment on ARROW-855 at 5/9/17 8:28 AM:
---------------------------------------------------------

{color:red}
environment:
scala:scala-sdk-2.10.4
jdk:1.7.0_79
arrow:arrow-0.1.0
the following is the test code: 
{color}   
 
 

import java.util.concurrent.ConcurrentHashMap
import org.apache.arrow.memory.{RootAllocator, BufferAllocator}
import org.apache.arrow.vector.NullableVarBinaryVector
import org.scalatest.{BeforeAndAfter, FunSuite}

import scala.util.Random



class ArrowLeakPureSuite extends FunSuite with BeforeAndAfter {
  val root: BufferAllocator = new RootAllocator(1024 * 1024 * 1024 * 8L)
  val elementCount = 1024
  val elementSize = 12800 // when 25600, all is ok.
  val binaGenerator = new BinaryGenerator(elementSize)

  def pushVector(allocator: BufferAllocator, name: String, length: Int): NullableVarBinaryVector = {
    val vector = new NullableVarBinaryVector(name, allocator)
    vector.allocateNew()
    val mutator = vector.getMutator
    (0 to length -1).foreach(i =>{
      val value = binaGenerator.next()
      mutator.setSafe(i, value, 0, value.length)
    })
    vector
  }

  test("leak") {
    val vectorCount = 100
    val entries = new ConcurrentHashMap[String, NullableVarBinaryVector]

    (0 to vectorCount).foreach(i => {
      val vector = pushVector(root,s"vector-$i", elementCount)
      entries.put(s"vector-$i", vector)
    })

    // Push Finished.
    println("Push Finished.")
    Thread.sleep( 10 * 1000)
    (0 to vectorCount).foreach(i => {
      val vector = entries.get(s"vector-$i")
      vector.close()
    })
{color:red}
    // Close Finished, now the data in Memory should be released, However it didn't when elementSize = 12800.
{color}
    println("Close Finished.")
    Thread.sleep( 20 * 1000)
    println("Test Finished.")
  }

  trait Generator[T] {

    def next(): T

    def nextStr(): String

  }

  abstract class GeneratorWithRandom[T] extends Generator[T] {

    val random = new Random()

    def nextStr() = next().toString
  }

  class BinaryGenerator(val length: Int) extends GeneratorWithRandom[Array[Byte]] {

    val binary = new Array[Byte](length)

    def next() = {
      random.nextBytes(binary)
      binary
    }

    override def nextStr() = {
      new String(binary)
    }
  }
}



was (Author: fushengxu):
{color:red}
environment:
scala:scala-sdk-2.10.4
jdk:1.7.0_79
arrow:arrow-0.1.0
{color}   
_ the following is the test code: _
 

import java.util.concurrent.ConcurrentHashMap
import org.apache.arrow.memory.{RootAllocator, BufferAllocator}
import org.apache.arrow.vector.NullableVarBinaryVector
import org.scalatest.{BeforeAndAfter, FunSuite}

import scala.util.Random



class ArrowLeakPureSuite extends FunSuite with BeforeAndAfter {
  val root: BufferAllocator = new RootAllocator(1024 * 1024 * 1024 * 8L)
  val elementCount = 1024
  val elementSize = 12800 // when 25600, all is ok.
  val binaGenerator = new BinaryGenerator(elementSize)

  def pushVector(allocator: BufferAllocator, name: String, length: Int): NullableVarBinaryVector = {
    val vector = new NullableVarBinaryVector(name, allocator)
    vector.allocateNew()
    val mutator = vector.getMutator
    (0 to length -1).foreach(i =>{
      val value = binaGenerator.next()
      mutator.setSafe(i, value, 0, value.length)
    })
    vector
  }

  test("leak") {
    val vectorCount = 100
    val entries = new ConcurrentHashMap[String, NullableVarBinaryVector]

    (0 to vectorCount).foreach(i => {
      val vector = pushVector(root,s"vector-$i", elementCount)
      entries.put(s"vector-$i", vector)
    })

    // Push Finished.
    println("Push Finished.")
    Thread.sleep( 10 * 1000)
    (0 to vectorCount).foreach(i => {
      val vector = entries.get(s"vector-$i")
      vector.close()
    })
  _  // Close Finished, now the data in Memory should be released, However it didn't when elementSize = 12800._
    println("Close Finished.")
    Thread.sleep( 20 * 1000)
    println("Test Finished.")
  }

  trait Generator[T] {

    def next(): T

    def nextStr(): String

  }

  abstract class GeneratorWithRandom[T] extends Generator[T] {

    val random = new Random()

    def nextStr() = next().toString
  }

  class BinaryGenerator(val length: Int) extends GeneratorWithRandom[Array[Byte]] {

    val binary = new Array[Byte](length)

    def next() = {
      random.nextBytes(binary)
      binary
    }

    override def nextStr() = {
      new String(binary)
    }
  }
}


> Arrow Memory Leak
> -----------------
>
>                 Key: ARROW-855
>                 URL: https://issues.apache.org/jira/browse/ARROW-855
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Java - Memory, Java - Vectors
>    Affects Versions: 0.1.0
>         Environment: CentOS release 6.7+Indellij IDEA
>            Reporter: xufusheng
>            Priority: Critical
>              Labels: test
>
> we create a memory table by arrow and the source data come from HBase.
> Create a memory table and then drop the table,there will be a memory leak.
> Hundreds of times,There will be OutOfMemoryError.
> anyone encounter similar problems?



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)