[ 
https://issues.apache.org/jira/browse/KAFKA-6343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alex Dunayevsky updated KAFKA-6343:
-----------------------------------
    Description: 
*Reproducing*: Create 5k topics *from the code* quickly, without any delays. 
Wait until brokers will finish loading them. This will actually never happen, 
since all brokers will go down after approx 10-15 minutes or more, depending on 
the hardware.

*Heap*: -Xmx/Xms: 5G, 10G, 50G, 256G... 
*Topology*: 3 brokers, 3 zk.

*Code for 5k topic creation:*
{code:java}
package kafka
import kafka.admin.AdminUtils
import kafka.utils.{Logging, ZkUtils}

object TestCreateTopics extends App with Logging {

  val zkConnect = "grid978:2185"
  var zkUtils = ZkUtils(zkConnect, 6000, 6000, isZkSecurityEnabled = false)

  for (topic <- 1 to 5000) {
    AdminUtils.createTopic(
      topic             = s"${topic.toString}",
      partitions        = 10,
      replicationFactor = 2,
      zkUtils           = zkUtils
    )
    logger.info(s"Created topic ${topic.toString}")
  }
}
{code}

*OOM:*
{code:java}
    java.io.IOException: Map failed
        at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:920)
        at kafka.log.AbstractIndex.<init>(AbstractIndex.scala:61)
        at kafka.log.OffsetIndex.<init>(OffsetIndex.scala:52)
        at kafka.log.LogSegment.<init>(LogSegment.scala:67)
        at kafka.log.Log.loadSegments(Log.scala:255)
        at kafka.log.Log.<init>(Log.scala:108)
        at kafka.log.LogManager.createLog(LogManager.scala:362)
        at kafka.cluster.Partition.getOrCreateReplica(Partition.scala:94)
        at 
kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
        at 
kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
        at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
        at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:174)
        at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:168)
        at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
        at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:242)
        at kafka.cluster.Partition.makeLeader(Partition.scala:168)
        at 
kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:758)
        at 
kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:757)
        at 
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
        at 
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
        at 
scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
        at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
        at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
        at kafka.server.ReplicaManager.makeLeaders(ReplicaManager.scala:757)
        at 
kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:703)
        at kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:148)
        at kafka.server.KafkaApis.handle(KafkaApis.scala:82)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.OutOfMemoryError: Map failed
        at sun.nio.ch.FileChannelImpl.map0(Native Method)
        at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:917)
        ... 28 more
{code}




  was:
Create 5k topics *from the code* quickly, without any delays. Wait until 
brokers will finish loading them. This will actually never happen, since all 
brokers will go down after approx 10-15 minutes or more, depending on the 
hardware.
*Heap*: -Xmx/Xms: 5G, 10G, 50G, 256G... 
*Topology*: 3 brokers, 3 zk.

*Code for 5k topic creation:*
{code:java}
package kafka
import kafka.admin.AdminUtils
import kafka.utils.{Logging, ZkUtils}

object TestCreateTopics extends App with Logging {

  val zkConnect = "grid978:2185"
  var zkUtils = ZkUtils(zkConnect, 6000, 6000, isZkSecurityEnabled = false)

  for (topic <- 1 to 5000) {
    AdminUtils.createTopic(
      topic             = s"${topic.toString}",
      partitions        = 10,
      replicationFactor = 2,
      zkUtils           = zkUtils
    )
    logger.info(s"Created topic ${topic.toString}")
  }
}
{code}

*OOM:*
{code:java}
    java.io.IOException: Map failed
        at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:920)
        at kafka.log.AbstractIndex.<init>(AbstractIndex.scala:61)
        at kafka.log.OffsetIndex.<init>(OffsetIndex.scala:52)
        at kafka.log.LogSegment.<init>(LogSegment.scala:67)
        at kafka.log.Log.loadSegments(Log.scala:255)
        at kafka.log.Log.<init>(Log.scala:108)
        at kafka.log.LogManager.createLog(LogManager.scala:362)
        at kafka.cluster.Partition.getOrCreateReplica(Partition.scala:94)
        at 
kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
        at 
kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
        at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
        at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:174)
        at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:168)
        at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
        at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:242)
        at kafka.cluster.Partition.makeLeader(Partition.scala:168)
        at 
kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:758)
        at 
kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:757)
        at 
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
        at 
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
        at 
scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
        at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
        at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
        at kafka.server.ReplicaManager.makeLeaders(ReplicaManager.scala:757)
        at 
kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:703)
        at kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:148)
        at kafka.server.KafkaApis.handle(KafkaApis.scala:82)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.OutOfMemoryError: Map failed
        at sun.nio.ch.FileChannelImpl.map0(Native Method)
        at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:917)
        ... 28 more
{code}





> OOM as the result of creation of 5k topics
> ------------------------------------------
>
>                 Key: KAFKA-6343
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6343
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.10.1.1
>         Environment: RHEL 7, RAM 755GB
>            Reporter: Alex Dunayevsky
>
> *Reproducing*: Create 5k topics *from the code* quickly, without any delays. 
> Wait until brokers will finish loading them. This will actually never happen, 
> since all brokers will go down after approx 10-15 minutes or more, depending 
> on the hardware.
> *Heap*: -Xmx/Xms: 5G, 10G, 50G, 256G... 
> *Topology*: 3 brokers, 3 zk.
> *Code for 5k topic creation:*
> {code:java}
> package kafka
> import kafka.admin.AdminUtils
> import kafka.utils.{Logging, ZkUtils}
> object TestCreateTopics extends App with Logging {
>   val zkConnect = "grid978:2185"
>   var zkUtils = ZkUtils(zkConnect, 6000, 6000, isZkSecurityEnabled = false)
>   for (topic <- 1 to 5000) {
>     AdminUtils.createTopic(
>       topic             = s"${topic.toString}",
>       partitions        = 10,
>       replicationFactor = 2,
>       zkUtils           = zkUtils
>     )
>     logger.info(s"Created topic ${topic.toString}")
>   }
> }
> {code}
> *OOM:*
> {code:java}
>     java.io.IOException: Map failed
>         at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:920)
>         at kafka.log.AbstractIndex.<init>(AbstractIndex.scala:61)
>         at kafka.log.OffsetIndex.<init>(OffsetIndex.scala:52)
>         at kafka.log.LogSegment.<init>(LogSegment.scala:67)
>         at kafka.log.Log.loadSegments(Log.scala:255)
>         at kafka.log.Log.<init>(Log.scala:108)
>         at kafka.log.LogManager.createLog(LogManager.scala:362)
>         at kafka.cluster.Partition.getOrCreateReplica(Partition.scala:94)
>         at 
> kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
>         at 
> kafka.cluster.Partition$$anonfun$4$$anonfun$apply$2.apply(Partition.scala:174)
>         at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
>         at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:174)
>         at kafka.cluster.Partition$$anonfun$4.apply(Partition.scala:168)
>         at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
>         at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:242)
>         at kafka.cluster.Partition.makeLeader(Partition.scala:168)
>         at 
> kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:758)
>         at 
> kafka.server.ReplicaManager$$anonfun$makeLeaders$4.apply(ReplicaManager.scala:757)
>         at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
>         at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:99)
>         at 
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
>         at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
>         at scala.collection.mutable.HashMap.foreach(HashMap.scala:99)
>         at kafka.server.ReplicaManager.makeLeaders(ReplicaManager.scala:757)
>         at 
> kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:703)
>         at 
> kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:148)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:82)
>         at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.OutOfMemoryError: Map failed
>         at sun.nio.ch.FileChannelImpl.map0(Native Method)
>         at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:917)
>         ... 28 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to