Replying to Jay's message, though some mailing list snafu made me not able to see it except through the archive. Apologies if it breaks threading:
I've confirmed that the Kafka process itself is running 64-bit, the information is included below. At this point I'm thinking it could be ulimit so we are running the tests again and keeping a close eye on that. Thanks, -Jeff 64-bit confirmation: [root@kafka03 bin]# ./jinfo -sysprops 3837 | grep sun.arch.data.model Attaching to process ID 3837, please wait... Debugger attached successfully. Server compiler detected. JVM version is 24.65-b04 sun.arch.data.model = 64 kafka02: Attaching to process ID 8632, please wait... Debugger attached successfully. Server compiler detected. JVM version is 24.65-b04 sun.arch.data.model = 64 kafka01: Attaching to process ID 4611, please wait... Debugger attached successfully. Server compiler detected. JVM version is 24.65-b04 sun.arch.data.model = 64 java -version Kafka 03: java version "1.7.0_67" Java(TM) SE Runtime Environment (build 1.7.0_67-b01) Java HotSpot(TM) 64-Bit Server VM (build 24.65-b04, mixed mode) Kafka02: java version "1.7.0_67" Java(TM) SE Runtime Environment (build 1.7.0_67-b01) Java HotSpot(TM) 64-Bit Server VM (build 24.65-b04, mixed mode) Kafka01: java version "1.7.0_67" Java(TM) SE Runtime Environment (build 1.7.0_67-b01) Java HotSpot(TM) 64-Bit Server VM (build 24.65-b04, mixed mode) Facter (OS architecture/kernel): Kafka 03: architecture => x86_64 hardwareisa => x86_64 hardwaremodel => x86_64 kernelrelease => 2.6.32-504.8.1.el6.x86_64 Kafka 02: architecture => x86_64 hardwareisa => x86_64 hardwaremodel => x86_64 kernelrelease => 2.6.32-504.8.1.el6.x86_64 Kafka 01: architecture => x86_64 hardwareisa => x86_64 hardwaremodel => x86_64 kernelrelease => 2.6.32-504.8.1.el6.x86_64 On Wed, May 13, 2015 at 5:00 PM, Jay Kreps <jay.kr...@gmail.com> wrote: I think "java.lang.OutOfMemoryError: Map failed" has usually been "out of address space for mmap" if memory serves. If you sum the length of all .index files while the service is running (not after stopped), do they sum to something really close to 2GB? If so it is likely either that the OS/arch is 32 bit (which on slack you said it wasn't) or possibly the jvm is in 32 bit mode? If you want to debug easiest test would be a simple program that did something like: public void static main(String[] args) throws Exception { RandomAccessFile raf = new RandomAccessFile("test-file-1", "rw"); RandomAccessFile raf2 = new RandomAccessFile("test-file-2", "rw"); raf1.setLength(2*1024*1024*1024); raf2.setLength(2*1024*1024*1024); MappedByteBuffer b1 = raf1.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, 2*1024*1024*1024); MappedByteBuffer b2 = raf2.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, 2*1024*1024*1024); } If you compile this and run with the same options you're running kafka with it should succeed but if it fails with the same error that is the address space limit for 32 bits kicking in. -Jay From: Jeff Field Sent: Wednesday, May 13, 2015 4:25 PM To: users@kafka.apache.org Subject: OutOfMemory error on broker when rolling logs Hello, We are doing a Kafka POC on our CDH cluster. We are running 3 brokers with 24TB (48TB Raw) of available RAID10 storage (XFS filesystem mounted with nobarrier/largeio) (HP Smart Array P420i for the controller, latest firmware) and 48GB of RAM. The broker is running with "-Xmx4G -Xms4G -server -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC". This is on RHEL 6.6 with the 2.6.32-504.8.1.el6.x86_64 kernel. JDK is jdk1.7.0_67 64-bit. We were using the 1.2.0 version of the Cloudera Kafka 0.8.2.0 build. We are upgrading to 1.3.0 after the RAID testing, but none of the fixes they included in 1.3.0 seem to be related to what we're seeing. We are using a custom producer to push copies of real messages from our existing messaging system onto Kafka in order to test ingestion rates and compression ratios. After a couple of hours (during which about 4.3 billion, ~2.2 terabytes before replication), one of our brokers will fail with an I/O error (2 slightly different ones so far) followed by a memory error. We're currently doing stress testing on the arrays (write/verify with IOzone set for 24 threads), but assuming the tests don't find anything on IO, what could cause this? Errors are included below. Thanks, -Jeff Occurrence 1: 2015-05-12 03:55:08,291 FATAL kafka.server.KafkaApis: [KafkaApi-834] Halting due to unrecoverable I/O error while handling produce request: kafka.common.KafkaStorageException: I/O exception in append to log 'TEST_TOPIC-1' at kafka.log.Log.append(Log.scala:266) at kafka.cluster.Partition$$anonfun$appendMessagesToLeader$1.apply(Partition.scala:379) at kafka.cluster.Partition$$anonfun$appendMessagesToLeader$1.apply(Partition.scala:365) at kafka.utils.Utils$.inLock(Utils.scala:561) at kafka.utils.Utils$.inReadLock(Utils.scala:567) at kafka.cluster.Partition.appendMessagesToLeader(Partition.scala:365) at kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:291) at kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:282) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at kafka.server.KafkaApis.appendToLocalLog(KafkaApis.scala:282) at kafka.server.KafkaApis.handleProducerOrOffsetCommitRequest(KafkaApis.scala:204) at kafka.server.KafkaApis.handle(KafkaApis.scala:59) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) at java.lang.Thread.run(Thread.java:745) Caused by: java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:888) at kafka.log.OffsetIndex.<init>(OffsetIndex.scala:74) at kafka.log.LogSegment.<init>(LogSegment.scala:57) at kafka.log.Log.roll(Log.scala:565) at kafka.log.Log.maybeRoll(Log.scala:539) at kafka.log.Log.append(Log.scala:306) ... 21 more Caused by: java.lang.OutOfMemoryError: Map failed at sun.nio.ch.FileChannelImpl.map0(Native Method) at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:885) ... 26 more Occurrence 2: 2015-05-12 20:08:15,052 FATAL kafka.server.KafkaApis: [KafkaApi-835] Halting due to unrecoverable I/O error while handling produce request: kafka.common.KafkaStorageException: I/O exception in append to log 'TEST_TOPIC-23' at kafka.log.Log.append(Log.scala:266) at kafka.cluster.Partition$$anonfun$appendMessagesToLeader$1.apply(Partition.scala:379) at kafka.cluster.Partition$$anonfun$appendMessagesToLeader$1.apply(Partition.scala:365) at kafka.utils.Utils$.inLock(Utils.scala:561) at kafka.utils.Utils$.inReadLock(Utils.scala:567) at kafka.cluster.Partition.appendMessagesToLeader(Partition.scala:365) at kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:291) at kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:282) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98) at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39) at scala.collection.mutable.HashMap.foreach(HashMap.scala:98) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at kafka.server.KafkaApis.appendToLocalLog(KafkaApis.scala:282) at kafka.server.KafkaApis.handleProducerOrOffsetCommitRequest(KafkaApis.scala:204) at kafka.server.KafkaApis.handle(KafkaApis.scala:59) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:59) at java.lang.Thread.run(Thread.java:745) Caused by: java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:888) at kafka.log.OffsetIndex.<init>(OffsetIndex.scala:74) at kafka.log.LogSegment.<init>(LogSegment.scala:57) at kafka.log.Log.roll(Log.scala:565) at kafka.log.Log.maybeRoll(Log.scala:539) at kafka.log.Log.append(Log.scala:306) ... 21 more Caused by: java.lang.OutOfMemoryError: Map failed at sun.nio.ch.FileChannelImpl.map0(Native Method) at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:885) ... 26 more Occurrence 3: 2015-05-13 01:11:14,626 FATAL kafka.server.ReplicaFetcherThread: [ReplicaFetcherThread-0-835], Disk error while replicating data. kafka.common.KafkaStorageException: I/O exception in append to log 'TEST_TOPIC-17' at kafka.log.Log.append(Log.scala:266) at kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:54) at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$1$$anonfun$apply$mcV$sp$2.apply(AbstractFetcherThread.scala:128) at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$1$$anonfun$apply$mcV$sp$2.apply(AbstractFetcherThread.scala:109) at scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:224) at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:403) at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:403) at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$1.apply$mcV$sp(AbstractFetcherThread.scala:109) at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$1.apply(AbstractFetcherThread.scala:109) at kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$1.apply(AbstractFetcherThread.scala:109) at kafka.utils.Utils$.inLock(Utils.scala:561) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:108) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:86) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:60) Caused by: java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:888) at kafka.log.OffsetIndex.<init>(OffsetIndex.scala:74) at kafka.log.LogSegment.<init>(LogSegment.scala:57) at kafka.log.Log.roll(Log.scala:565) at kafka.log.Log.maybeRoll(Log.scala:539) at kafka.log.Log.append(Log.scala:306) ... 13 more Caused by: java.lang.OutOfMemoryError: Map failed at sun.nio.ch.FileChannelImpl.map0(Native Method) at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:885) ... 18 more