[ https://issues.apache.org/jira/browse/CASSANDRA-14978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16954562#comment-16954562 ]
ykalfon commented on CASSANDRA-14978: ------------------------------------- I can confirm that also: *disk_access_mode=mmap_index_only* solve this issue. > Cassandra going down with "java.lang.OutOfMemoryError: Map failed" and "LEAK > DETECTED" > -------------------------------------------------------------------------------------- > > Key: CASSANDRA-14978 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14978 > Project: Cassandra > Issue Type: Bug > Reporter: Yakir Gibraltar > Priority: Normal > > Cassandra version: 3.11.4 > OS: CentOS Linux release 7.4.1708 (Core) > Kernel: 3.10.0-957.10.1.el7.x86_64 > JDK: jdk1.8.0_131 > Heap: same errors with 16GB / 32GB / 64GB. > *We are seeing this errors in production:* > *java.io.IOException: Map failed:* > {code:java} > ERROR [CompactionExecutor:5017] 2019-01-14 00:02:04,763 > CassandraDaemon.java:228 - Exception in thread > Thread[CompactionExecutor:5017,1,main] > org.apache.cassandra.io.FSReadError: java.io.IOException: Map failed > at > org.apache.cassandra.io.util.ChannelProxy.map(ChannelProxy.java:157) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.MmappedRegions$State.add(MmappedRegions.java:310) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.MmappedRegions$State.access$400(MmappedRegions.java:246) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.MmappedRegions.updateState(MmappedRegions.java:181) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.MmappedRegions.<init>(MmappedRegions.java:73) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.MmappedRegions.<init>(MmappedRegions.java:61) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.MmappedRegions.map(MmappedRegions.java:104) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.util.FileHandle$Builder.complete(FileHandle.java:362) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.sstable.format.big.BigTableWriter.openEarly(BigTableWriter.java:290) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.sstable.SSTableRewriter.maybeReopenEarly(SSTableRewriter.java:179) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:134) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter.realAppend(DefaultCompactionWriter.java:65) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:142) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:201) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:85) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:274) > ~[apache-cassandra-3.11.3.jar:3.11.3] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_131] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ~[na:1.8.0_131] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > ~[na:1.8.0_131] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_131] > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) > [apache-cassandra-3.11.3.jar:3.11.3] > at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131] > Caused by: java.io.IOException: Map failed > at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:940) > ~[na:1.8.0_131] > at > org.apache.cassandra.io.util.ChannelProxy.map(ChannelProxy.java:153) > ~[apache-cassandra-3.11.3.jar:3.11.3] > ... 23 common frames omitted > Caused by: java.lang.OutOfMemoryError: Map failed > at sun.nio.ch.FileChannelImpl.map0(Native Method) ~[na:1.8.0_131] > at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:937) > ~[na:1.8.0_131] > ... 24 common frames omitted > {code} > *LEAK DETECTED error:* > {code:java} > ERROR [Reference-Reaper:1] 2019-01-14 00:03:46,469 Ref.java:224 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@6a4ef142) to class > org.apache.cassandra.io.util.SafeMemory$MemoryTidy@1651696741:Memory@[6b91a27c5290..6b91a27de290) > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2019-01-14 00:03:46,520 Ref.java:224 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@6c458f8a) to class > org.apache.cassandra.io.util.FileHandle$Cleanup@1179238225:/var/lib/cassandra/data/disk1/sessions_rawdata/sessions_v2_2019_01_13-19be8e90037011e9a45847402874bbd7/mc-1209-big-Index.db > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2019-01-14 00:03:46,520 Ref.java:224 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@5b90823b) to class > org.apache.cassandra.io.util.MmappedRegions$Tidier@783549664:/var/lib/cassandra/data/disk1/sessions_rawdata/sessions_v2_2019_01_13-19be8e90037011e9a45847402874bbd7/mc-1209-big-Data.db > was not released before the reference was garbage collected > ERROR [Reference-Reaper:1] 2019-01-14 00:03:46,520 Ref.java:224 - LEAK > DETECTED: a reference > (org.apache.cassandra.utils.concurrent.Ref$State@6ecdf763) to class > org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$Tidy@1710583516:[Memory@[0..3e24), > Memory@[0..45e88)] was not released before the reference was garbage > collected{code} > > *Limits of Cassandra process:* > {code:java} > [root@cass063 ~ ]# cat /proc/`ps -ef | grep CassandraDaemon | grep -v grep | > awk '\{print $2}'`/limits > Limit Soft Limit Hard Limit Units > Max cpu time unlimited unlimited seconds > Max file size unlimited unlimited bytes > Max data size unlimited unlimited bytes > Max stack size 8388608 unlimited bytes > Max core file size 0 unlimited bytes > Max resident set unlimited unlimited bytes > Max processes 32768 32768 processes > Max open files 100000 100000 files > Max locked memory unlimited unlimited bytes > Max address space unlimited unlimited bytes > Max file locks unlimited unlimited locks > Max pending signals 766985 766985 signals > Max msgqueue size 819200 819200 bytes > Max nice priority 0 0 > Max realtime priority 0 0 > Max realtime timeout unlimited unlimited us{code} > > * vm.max_map_count parameter on OS:* > {code:java} > [root@cass063 ~]# sysctl vm.max_map_count > vm.max_map_count = 1073741824 > {code} > > *cassandra.yaml:* > {code:java} > cluster_name: 'Cass Cluster' > num_tokens: 256 > hinted_handoff_enabled: false > max_hint_window_in_ms: 10800000 > hinted_handoff_throttle_in_kb: 1024 > max_hints_delivery_threads: 2 > hints_directory: /var/lib/cassandra/hints > hints_flush_period_in_ms: 10000 > max_hints_file_size_in_mb: 128 > batchlog_replay_throttle_in_kb: 1024 > authenticator: AllowAllAuthenticator > authorizer: AllowAllAuthorizer > role_manager: CassandraRoleManager > roles_validity_in_ms: 2000 > permissions_validity_in_ms: 2000 > credentials_validity_in_ms: 2000 > partitioner: org.apache.cassandra.dht.Murmur3Partitioner > data_file_directories: > - /var/lib/cassandra/data/disk1 > commitlog_directory: /var/lib/cassandra/data/disk1/commitlog > cdc_enabled: false > disk_failure_policy: stop > commit_failure_policy: stop > prepared_statements_cache_size_mb: > thrift_prepared_statements_cache_size_mb: > key_cache_size_in_mb: 0 > key_cache_save_period: 3600 > row_cache_size_in_mb: 0 > row_cache_save_period: 0 > counter_cache_size_in_mb: > counter_cache_save_period: 7200 > saved_caches_directory: /var/lib/cassandra/data/disk1/saved_caches > commitlog_sync: periodic > commitlog_sync_period_in_ms: 10000 > commitlog_segment_size_in_mb: 32 > seed_provider: > - class_name: org.apache.cassandra.locator.SimpleSeedProvider > parameters: > - seeds: "10.110.30.1,10.110.30.2,10.110.30.3" > concurrent_reads: 48 > concurrent_writes: 96 > concurrent_counter_writes: 32 > concurrent_materialized_view_writes: 32 > file_cache_size_in_mb: 10240 > memtable_offheap_space_in_mb: 10240 > memtable_cleanup_threshold: 0.1 > memtable_allocation_type: offheap_buffers > commitlog_total_space_in_mb: 8192 > memtable_flush_writers: 8 > index_summary_capacity_in_mb: > index_summary_resize_interval_in_minutes: 60 > trickle_fsync: true > trickle_fsync_interval_in_kb: 10240 > storage_port: 7000 > ssl_storage_port: 7001 > listen_address: 10.106.62.34 > start_native_transport: true > native_transport_port: 9042 > start_rpc: false > rpc_address: 0.0.0.0 > rpc_port: 9160 > broadcast_rpc_address: 10.106.62.34 > rpc_keepalive: true > rpc_server_type: hsha > rpc_max_threads: 128 > thrift_framed_transport_size_in_mb: 15 > incremental_backups: false > snapshot_before_compaction: false > auto_snapshot: true > column_index_size_in_kb: 64 > column_index_cache_size_in_kb: 2 > concurrent_compactors: 32 > compaction_throughput_mb_per_sec: 500 > sstable_preemptive_open_interval_in_mb: 50 > stream_throughput_outbound_megabits_per_sec: 0 > read_request_timeout_in_ms: 10000 > range_request_timeout_in_ms: 10000 > write_request_timeout_in_ms: 60000 > counter_write_request_timeout_in_ms: 10000 > cas_contention_timeout_in_ms: 1000 > truncate_request_timeout_in_ms: 60000 > request_timeout_in_ms: 10000 > slow_query_log_timeout_in_ms: 500 > cross_node_timeout: false > phi_convict_threshold: 12 > endpoint_snitch: GossipingPropertyFileSnitch > dynamic_snitch_update_interval_in_ms: 100 > dynamic_snitch_reset_interval_in_ms: 600000 > dynamic_snitch_badness_threshold: 0.5 > request_scheduler: org.apache.cassandra.scheduler.NoScheduler > server_encryption_options: > internode_encryption: none > keystore: conf/.keystore > keystore_password: cassandra > truststore: conf/.truststore > truststore_password: cassandra > client_encryption_options: > enabled: false > optional: false > keystore: conf/.keystore > keystore_password: cassandra > internode_compression: dc > inter_dc_tcp_nodelay: false > tracetype_query_ttl: 86400 > tracetype_repair_ttl: 604800 > enable_user_defined_functions: false > enable_scripted_user_defined_functions: false > enable_materialized_views: true > windows_timer_interval: 1 > transparent_data_encryption_options: > enabled: false > chunk_length_kb: 64 > cipher: AES/CBC/PKCS5Padding > key_alias: testing:1 > key_provider: > - class_name: org.apache.cassandra.security.JKSKeyProvider > parameters: > - keystore: conf/.keystore > keystore_password: cassandra > store_type: JCEKS > key_password: cassandra > tombstone_warn_threshold: 1000 > tombstone_failure_threshold: 100000 > batch_size_warn_threshold_in_kb: 5 > batch_size_fail_threshold_in_kb: 50 > unlogged_batch_across_partitions_warn_threshold: 10 > compaction_large_partition_warning_threshold_mb: 10 > gc_warn_threshold_in_ms: 1000 > back_pressure_enabled: false > back_pressure_strategy: > - class_name: org.apache.cassandra.net.RateBasedBackPressure > parameters: > - high_ratio: 0.90 > factor: 5 > flow: FAST{code} > > *A lot of maps, 200K maps of cassandra process,*: > {code:java} > [root@cass063 ~]# wc -l /proc/`ps -ef | grep CassandraDaemon | grep -v grep | > awk '{print $2}'`/maps > 239587 /proc/202664/maps{code} > I got same error with heap of 16GB / 32GB / 64GB. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org