Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Large aggregation query -> OOM -> Index corruption -> data loss #12041

Closed
kristoffer-dyrkorn opened this issue Jul 6, 2015 · 11 comments
Closed
Labels
:Core/Infra/Core Core issues without another label discuss feedback_needed

Comments

@kristoffer-dyrkorn
Copy link

We have experienced data loss in ES (1.5.2) after running very heavy aggregation queries. We get OOMs (which is not ideal, but tolerable, given our context) but also index corruption and data loss (which is bad).

Any ideas as to what has happened here - and how this could be prevented? Please see the log excerpt:

TIMESTAMP="2015-06-29 11:55:49,689" LEVEL="WARN" THREAD="[elasticsearch[server05][search][T#20]]" HOST="server05" LOGGER="index.search.slowlog.query" MESSAGE="[server05] [ua_log_domain_event_2015_06][4] took[27.1m], took_millis[1626309], types[], stats[], search_type[COUNT], total_shards[12], source[(removed)], extra_source[], "
TIMESTAMP="2015-06-29 11:55:49,735" LEVEL="WARN" THREAD="[elasticsearch[server05][scheduler][T#1]]" HOST="server05" LOGGER="monitor.jvm" MESSAGE="[server05] [gc][old][266076][102] duration [18.7m], collections [69]/[19.9m], total [18.7m]/[25.8m], memory [7.9gb]->[7.8gb]/[7.9gb], all_pools {[young] [532.5mb]->[532.5mb]/[532.5mb]}{[survivor] [65.2mb]->[16.5mb]/[66.5mb]}{[old] [7.3gb]->[7.3gb]/[7.3gb]}"
TIMESTAMP="2015-06-29 11:55:49,752" LEVEL="WARN" THREAD="[elasticsearch[server05][http_server_boss][T#1]{New I/O server boss #51}]" HOST="server05" LOGGER="netty.channel.socket.nio.AbstractNioSelector" MESSAGE="Failed to accept a connection."
java.lang.OutOfMemoryError: Java heap space
TIMESTAMP="2015-06-29 11:55:49,752" LEVEL="WARN" THREAD="[elasticsearch[server05][management][T#4]]" HOST="server05" LOGGER="transport.netty" MESSAGE="[server05] Failed to send error message back to client for action [indices:monitor/stats[s]]"
java.lang.OutOfMemoryError: Java heap space
TIMESTAMP="2015-06-29 11:55:49,773" LEVEL="WARN" THREAD="[elasticsearch[server05][[http_server_worker.default]][T#1]{New I/O worker #18}]" HOST="server05" LOGGER="netty.channel.socket.nio.AbstractNioSelector" MESSAGE="Unexpected exception in the selector loop."
java.lang.OutOfMemoryError: Java heap space
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:268) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:255) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42) [elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
TIMESTAMP="2015-06-29 11:55:49,774" LEVEL="WARN" THREAD="[elasticsearch[server05][[http_server_worker.default]][T#5]{New I/O worker #22}]" HOST="server05" LOGGER="transport.netty" MESSAGE="[server05] exception caught on transport layer [[id: 0x99ba54a2, /10.247.21.4:33540 => /10.247.20.5:9300]], closing connection"
java.lang.OutOfMemoryError: Java heap space
    at org.elasticsearch.common.io.stream.CachedStreamInput.instance(CachedStreamInput.java:48) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.io.stream.CachedStreamInput.cachedHandles(CachedStreamInput.java:62) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.transport.netty.MessageChannelHandler.messageReceived(MessageChannelHandler.java:109) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:296) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:462) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:443) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.OpenChannelsHandler.handleUpstream(OpenChannelsHandler.java:74) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:268) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:255) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42) [elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
TIMESTAMP="2015-06-29 11:55:49,774" LEVEL="WARN" THREAD="[elasticsearch[server05][management][T#4]]" HOST="server05" LOGGER="transport.netty" MESSAGE="[server05] Actual Exception"
java.lang.OutOfMemoryError: Java heap space
    at java.io.UnixFileSystem.resolve(UnixFileSystem.java:108) ~[na:1.8.0_31]
    at java.io.File.<init>(File.java:367) ~[na:1.8.0_31]
    at org.apache.lucene.store.FSDirectory$1.accept(FSDirectory.java:221) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at java.io.File.list(File.java:1161) ~[na:1.8.0_31]
    at org.apache.lucene.store.FSDirectory.listAll(FSDirectory.java:218) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FSDirectory.listAll(FSDirectory.java:237) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.elasticsearch.index.store.fs.DefaultFsDirectoryService$1.listAll(DefaultFsDirectoryService.java:57) ~[elasticsearch-1.5.2.jar:na]
    at org.apache.lucene.store.FilterDirectory.listAll(FilterDirectory.java:48) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FilterDirectory.listAll(FilterDirectory.java:48) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FilterDirectory.listAll(FilterDirectory.java:48) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.elasticsearch.index.store.Store$StoreStatsCache.estimateSize(Store.java:1464) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store$StoreStatsCache.refresh(Store.java:1456) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store$StoreStatsCache.refresh(Store.java:1443) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.util.SingleObjectCache.getOrRefresh(SingleObjectCache.java:55) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store.stats(Store.java:285) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.shard.IndexShard.storeStats(IndexShard.java:600) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.admin.indices.stats.CommonStats.<init>(CommonStats.java:139) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.admin.indices.stats.ShardStats.<init>(ShardStats.java:49) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction.shardOperation(TransportIndicesStatsAction.java:208) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction.shardOperation(TransportIndicesStatsAction.java:56) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction$ShardTransportHandler.messageReceived(TransportBroadcastOperationAction.java:339) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction$ShardTransportHandler.messageReceived(TransportBroadcastOperationAction.java:325) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.transport.netty.MessageChannelHandler$RequestHandler.doRun(MessageChannelHandler.java:277) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:36) ~[elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
TIMESTAMP="2015-06-29 11:55:49,783" LEVEL="WARN" THREAD="[elasticsearch[server05][generic][T#95]]" HOST="server05" LOGGER="index.store" MESSAGE="[server05] [metrics-2015-06][0] failed to build store metadata. checking segment info integrity (with commit [no])"
java.nio.file.NoSuchFileException: /data/project/nodes/0/indices/metrics-2015-06/0/index/_2wk2.si
    at sun.nio.fs.UnixException.translateToIOException(UnixException.java:86) ~[na:1.8.0_31]
    at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) ~[na:1.8.0_31]
    at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) ~[na:1.8.0_31]
    at sun.nio.fs.UnixFileSystemProvider.newFileChannel(UnixFileSystemProvider.java:177) ~[na:1.8.0_31]
    at java.nio.channels.FileChannel.open(FileChannel.java:287) ~[na:1.8.0_31]
    at java.nio.channels.FileChannel.open(FileChannel.java:335) ~[na:1.8.0_31]
    at org.apache.lucene.store.NIOFSDirectory.openInput(NIOFSDirectory.java:81) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FileSwitchDirectory.openInput(FileSwitchDirectory.java:172) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FilterDirectory.openInput(FilterDirectory.java:80) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FilterDirectory.openInput(FilterDirectory.java:80) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.store.FilterDirectory.openInput(FilterDirectory.java:80) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.elasticsearch.index.store.Store$StoreDirectory.openInput(Store.java:683) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store$MetadataSnapshot.checksumFromLuceneFile(Store.java:892) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store$MetadataSnapshot.buildMetadata(Store.java:791) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store$MetadataSnapshot.<init>(Store.java:768) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store.getMetadata(Store.java:222) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.store.Store.getMetadataOrEmpty(Store.java:185) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.listStoreMetaData(TransportNodesListShardStoreMetaData.java:160) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.nodeOperation(TransportNodesListShardStoreMetaData.java:141) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.nodeOperation(TransportNodesListShardStoreMetaData.java:62) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.support.nodes.TransportNodesOperationAction$NodeTransportHandler.messageReceived(TransportNodesOperationAction.java:278) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.support.nodes.TransportNodesOperationAction$NodeTransportHandler.messageReceived(TransportNodesOperationAction.java:269) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.transport.netty.MessageChannelHandler$RequestHandler.doRun(MessageChannelHandler.java:277) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:36) [elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
TIMESTAMP="2015-06-29 11:55:49,774" LEVEL="WARN" THREAD="[elasticsearch[server05][[http_server_worker.default]][T#9]{New I/O worker #26}]" HOST="server05" LOGGER="netty.channel.socket.nio.AbstractNioSelector" MESSAGE="Unexpected exception in the selector loop."
java.lang.OutOfMemoryError: Java heap space
TIMESTAMP="2015-06-29 11:55:49,808" LEVEL="WARN" THREAD="[elasticsearch[server05][[http_server_worker.default]][T#5]{New I/O worker #22}]" HOST="server05" LOGGER="transport.netty" MESSAGE="[server05] exception caught on transport layer [[id: 0x99ba54a2, /10.247.21.4:33540 :> /10.247.20.5:9300]], closing connection"
java.io.StreamCorruptedException: invalid internal transport message format, got (1d,63,6c,75)
    at org.elasticsearch.transport.netty.SizeHeaderFrameDecoder.decode(SizeHeaderFrameDecoder.java:47) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:425) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.OpenChannelsHandler.handleUpstream(OpenChannelsHandler.java:74) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:268) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.Channels.fireMessageReceived(Channels.java:255) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:108) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42) [elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
TIMESTAMP="2015-06-29 11:55:49,818" LEVEL="INFO" THREAD="[elasticsearch[server05][generic][T#95]]" HOST="server05" LOGGER="index.store" MESSAGE="[server05] [metrics-2015-06][0] Failed to open / find files while reading metadata snapshot"
@bleskes
Copy link
Contributor

bleskes commented Jul 6, 2015

This message is worrisome - it seems you have 7GB of memory, which took 18.7 minutes to GC? What kind of hardware are you running on? did you disable swap?

[server05] [gc][old][266076][102] duration [18.7m], collections [69]/[19.9m], total [18.7m]/[25.8m], memory [7.9gb]->[7.8gb]/[7.9gb], all_pools {[young] [532.5mb]->[532.5mb]/[532.5mb]}{[survivor] [65.2mb]->[16.5mb]/[66.5mb]}{[old] [7.3gb]->[7.3gb]/[7.3gb]}"

Once you've reached an OOM you can better restart the node as many unexpected things can happen. It's especially bad of the oom happened in the network stack (like yours did). Can you specify some more details about the corruption reported?

@kristoffer-dyrkorn
Copy link
Author

Each data node (like the one above) in our cluster has 16 GB RAM and 8 GB heap. The aggregations are very heavy, so long GCs are as such not the main problem here (we need to find ways to limit the queries on our part).

My main question here is: Should an OOM occur on one node, would shard corruption - and then data loss - then be one of the possible (expected) outcomes?

The net effect of the crash has been a data loss of 20% of the original data for a specific set of indices. By looking at shard information we see that some shards have less data (take less space on disk) than they should. We also see missing documents in the affected indices - very evenly spread out across our primary key (an event number id - our data are time series).

We suspect that due to OOM some shards have been corrupted and then discarded. We have 1 replica for each shard (that is, 1 primary and 1 replica) in the cluster but this has not prevented data loss. (Yes, we could have had more replicas but that is not the main issue here.)

It seems that long GC pauses have made data nodes unable to connect to the master (we see traces of that in the logs on the data nodes) so an additional factor here might be that the master has not had a chance to become aware of the shard corruption.

Mlockall is set to true by us (in elasticsearch.yml) but reading out the actual value from /_nodes/process shows the actual value is false. We are alredy aware of that - we believe the reason is that we have to use virtual machines on VMWare.

We have not disabled swap explicity, so we run on OS defaults. We are using RHEL (7.0, I think).

@bleskes
Copy link
Contributor

bleskes commented Jul 6, 2015

The net effect of the crash has been a data loss of 20% of the original data for a specific set of indices. By looking at shard information we see that some shards have less data (take less space on disk) than they should.

How much less? Shards may very in size based on their merge schedules.

We also see missing documents in the affected indices - very evenly spread out across our primary key (an event number id - our data are time series).

Do you delete documents ? if so, you should take them into account as the stats report the total amount of documents in the index. It has a separate entry for deleted docs (which will be purge away by the background merge process).

Are there any other signs of corruption? A log entry would be great.

It seems that long GC pauses have made data nodes unable to connect to the master (we see traces of that in the logs on the data nodes) so an additional factor here might be that the master has not had a chance to become aware of the shard corruption.

A non responsive will be removed from the cluster and it's shard will be assigned somewhere else. Once the node rejoins (after the GC finishes) it will be reassigned shards, which will be resynced to the primaries.

@kristoffer-dyrkorn
Copy link
Author

How much less? Shards may very in size based on their merge schedules.

The indices are time-partitioned (one index for each month of data). Each index has 6 shards. The distribution of data for the May index - after our problems - was as follows:

Shard 0, 2, 3, 4, 5: 4.1-4.2 Gb
Shard 1: 650-660 Mb (primary & replica)

For the February index, the distribution was:

Shard 1: 115 bytes / 79 bytes (primary & replica)
Shard 0, 2, 3, 4, 5: 0.95 - 1.1 Gb

For other indices (not affected by data loss) we see very even shard sizes, within 5% of each other. So we suspect merge schedules are not a factor here.

Do you delete documents ?

No. This is mostly an append-only system. At times we update existing documents (which we realise may be implemented internally in Lucene as delete + add), but also here we do not think the differences are due to deletes (i.e. updates), as the indices that not were affected by data loss look very similar to each other and they are all quite different to the ones affected.

Are there any other signs of corruption? A log entry would be great.

This is for us the the main sign of corruption:

java.nio.file.NoSuchFileException: /data/project/nodes/0/indices/metrics-2015-06/0/index/_2wk2.si

I will look for other log messages and update this issue.

We see problems on other data nodes as well (OOMs, missing cluster synchronisation) during the failure period. Here is an example of events from a different node.

First occurence of an OOM:

TIMESTAMP="2015-06-29 12:08:42,117" LEVEL="WARN" THREAD="[elasticsearch[server06][refresh][T#3]]" HOST="server06" LOGGER="index.shard" MESSAGE="[server06] [event_2015_06][4] Failed to perform scheduled engine refresh"
org.elasticsearch.index.engine.RefreshFailedEngineException: [event_2015_06][4] Refresh failed
    at org.elasticsearch.index.engine.InternalEngine.refresh(InternalEngine.java:576) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.shard.IndexShard.refresh(IndexShard.java:565) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.shard.IndexShard$EngineRefresher$1.run(IndexShard.java:1095) ~[elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
java.lang.OutOfMemoryError: Java heap space
TIMESTAMP="2015-06-29 12:08:42,117" LEVEL="WARN" THREAD="[elasticsearch[server06][flush][T#2]]" HOST="server06" LOGGER="index.engine" MESSAGE="[server06] [event_2015_06][4] failed engine [lucene commit failed]"
org.apache.lucene.store.AlreadyClosedException: this IndexWriter is closed
    at org.apache.lucene.index.IndexWriter.ensureOpen(IndexWriter.java:700) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.index.IndexWriter.ensureOpen(IndexWriter.java:714) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.index.IndexWriter.commit(IndexWriter.java:3100) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.elasticsearch.index.engine.InternalEngine.commitIndexWriter(InternalEngine.java:1226) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.engine.InternalEngine.flush(InternalEngine.java:637) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.engine.InternalEngine.flush(InternalEngine.java:593) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.shard.IndexShard.flush(IndexShard.java:675) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.translog.TranslogService$TranslogBasedFlush$1.run(TranslogService.java:203) [elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
Caused by: java.lang.OutOfMemoryError: Java heap space

The node is then unable to let the master know of the shard failure:

TIMESTAMP="2015-06-29 12:10:32,819" LEVEL="WARN" THREAD="[elasticsearch[server06][clusterService#updateTask][T#1]]" HOST="server06" LOGGER="indices.cluster" MESSAGE="[server06] [[event_2015_06][4]] marking and sending shard failed due to [master [null] marked shard as started, but shard has not been created, mark shard as failed]"

The node is then unable to create shards:

TIMESTAMP="2015-06-29 12:15:44,793" LEVEL="WARN" THREAD="[elasticsearch[server06][clusterService#updateTask][T#1]]" HOST="server06" LOGGER="indices.cluster" MESSAGE="[server06] [[event_2015_06][0]] marking and sending shard failed due to [failed to create shard]"
org.elasticsearch.index.shard.IndexShardCreationException: [event_2015_06][0] failed to create shard
    at org.elasticsearch.index.IndexService.createShard(IndexService.java:357) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.indices.cluster.IndicesClusterStateService.applyInitializingShard(IndicesClusterStateService.java:699) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.indices.cluster.IndicesClusterStateService.applyNewOrUpdatedShards(IndicesClusterStateService.java:600) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.indices.cluster.IndicesClusterStateService.clusterChanged(IndicesClusterStateService.java:183) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.cluster.service.InternalClusterService$UpdateTask.run(InternalClusterService.java:467) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.runAndClean(PrioritizedEsThreadPoolExecutor.java:188) [elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.run(PrioritizedEsThreadPoolExecutor.java:158) [elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
Caused by: org.apache.lucene.store.LockObtainFailedException: Can't lock shard [event_2015_06][0], timed out after 5000ms
    at org.elasticsearch.env.NodeEnvironment$InternalShardLock.acquire(NodeEnvironment.java:415) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.env.NodeEnvironment.shardLock(NodeEnvironment.java:343) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.IndexService.createShard(IndexService.java:310) ~[elasticsearch-1.5.2.jar:na]
    ... 9 common frames omitted

Once the node rejoins (after the GC finishes) it will be reassigned shards, which will be resynced
to the primaries.

We experienced problems with reassignments so we had to allocate shards manually to get ES out of its "red state". It seems we hit the case described here:

#11309

At least, this comment fits very well with our hypothesis (and previous observations):

I suspect that the same behavior will occur if both primary and replica shards have index
corruptions.

Still, my main question is: Should an OOM occur on one node, would shard corruption - and then data loss - then be one of the possible (expected) outcomes?

It would be very valuable for us to have your opinion on whether data loss (on a single node) could be expected should an OOM occur.

@kristoffer-dyrkorn
Copy link
Author

Searching ES logs for information about the February index (where some shards were emptied) gave the following result:

TIMESTAMP="2015-06-29 13:18:56,511" LEVEL="WARN" THREAD="[elasticsearch[server03][generic][T#5]]" HOST="server03" LOGGER="indices.cluster" MESSAGE="[server03] [[event_2015_02][1]] marking and sending shard failed due to [failed recovery]"
org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException: [event_2015_02][1] failed to fetch index version after copying it over
    at org.elasticsearch.index.gateway.local.LocalIndexShardGateway.recover(LocalIndexShardGateway.java:157) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.gateway.IndexShardGatewayService$1.run(IndexShardGatewayService.java:112) ~[elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
Caused by: org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException: [event_2015_02][1] shard allocated for local recovery (post api), should exist, but doesn't, current files: []
    at org.elasticsearch.index.gateway.local.LocalIndexShardGateway.recover(LocalIndexShardGateway.java:131) ~[elasticsearch-1.5.2.jar:na]
    ... 4 common frames omitted
Caused by: org.apache.lucene.index.IndexNotFoundException: no segments* file found in store(least_used[rate_limited(default(mmapfs(/data/project/nodes/0/indices/event_2015_02/1/index),niofs(/data/project/nodes/0/indices/event_2015_02/1/index)), type=MERGE, rate=20.0)]): files: []
    at org.apache.lucene.index.SegmentInfos$FindSegmentsFile.run(SegmentInfos.java:881) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.index.SegmentInfos$FindSegmentsFile.run(SegmentInfos.java:769) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.apache.lucene.index.SegmentInfos.read(SegmentInfos.java:458) ~[lucene-core-4.10.4.jar:4.10.4 1662817 - mike - 2015-02-27 16:38:43]
    at org.elasticsearch.common.lucene.Lucene.readSegmentInfos(Lucene.java:89) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.index.gateway.local.LocalIndexShardGateway.recover(LocalIndexShardGateway.java:122) ~[elasticsearch-1.5.2.jar:na]
    ... 4 common frames omitted

This is the first occurrence of log entry from a data node that has handled those shards (within the period we had failures). So it seems at this point in time, the directory containing this shard (shard 1) has been unexpectedly emptied.

A bit before this, we see that the master node cannot contact this data node. These two log entries (one above, one below) are the only ones that show up in our log analysis tool and that give hints on what happened to the shards.

TIMESTAMP="2015-06-29 12:28:09,173" LEVEL="WARN" THREAD="[elasticsearch[server01][clusterService#updateTask][T#1]]" HOST="server01" LOGGER="gateway.local" MESSAGE="[server01] [event_2015_02][1]: failed to list shard stores on node [o7gM5FhPRSKUVqQpxPHq5A]"
org.elasticsearch.action.FailedNodeException: Failed node [o7gM5FhPRSKUVqQpxPHq5A]
    at org.elasticsearch.action.support.nodes.TransportNodesOperationAction$AsyncAction.onFailure(TransportNodesOperationAction.java:206) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.support.nodes.TransportNodesOperationAction$AsyncAction.access$1000(TransportNodesOperationAction.java:97) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.action.support.nodes.TransportNodesOperationAction$AsyncAction$4.handleException(TransportNodesOperationAction.java:178) ~[elasticsearch-1.5.2.jar:na]
    at org.elasticsearch.transport.TransportService$TimeoutHandler.run(TransportService.java:529) ~[elasticsearch-1.5.2.jar:na]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_31]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_31]
    at java.lang.Thread.run(Thread.java:745) [na:1.8.0_31]
Caused by: org.elasticsearch.transport.ReceiveTimeoutTransportException: [server04][inet[/10.247.21.4:9300]][internal:cluster/nodes/indices/shard/store[n]] request_id [3857481] timed out after [30000ms]
    ... 4 common frames omitted

@bleskes
Copy link
Contributor

bleskes commented Jul 23, 2015

Still, my main question is: Should an OOM occur on one node, would shard corruption - and then data loss - then be one of the possible (expected) outcomes?

OOM should not cause data corruption and certainly not shards to be removed.

Is the following something that you can reproduce/happens often? I would love to have some trace logs enabled and debug further...

Caused by: org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException: [event_2015_02][1] shard allocated for local recovery (post api), should exist, but doesn't, current files: []

@clintongormley clintongormley added discuss :Core/Infra/Core Core issues without another label labels Jul 24, 2015
@kristoffer-dyrkorn
Copy link
Author

We are working on reproducing the issue now, will post further details here.

@clintongormley
Copy link
Contributor

@kristoffer-dyrkorn this may be related to the issue fixed in #12487 so I would advise upgrading.

@kristoffer-dyrkorn
Copy link
Author

We have not been able to recreate the issue and cannot spend further resources on trying. So I am afraid we cannot reach a final conclusion on what went wrong.

We will however upgrade now.

@clintongormley
Copy link
Contributor

thanks @kristoffer-dyrkorn - closing for now.

@nilsga
Copy link

nilsga commented Sep 4, 2015

This issue happende again today (still on 1.5.2), with the same sequence of errors as described earlier. It does indeed look very similar to #12487. We lost four shards (2 x primary and replica) in this crash, and both primary and replica were located on the nodes that went OOM. After nodes had been restarted, the files and directories for shard 4 and 5 were no longer on disk on the affected data nodes. Both nodes had long GC pauses, so I guess there were good chances for a race condition to happen. Luckily, it only affected one of the indices, which are backed up nightly, so we were able to restore all the data.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
:Core/Infra/Core Core issues without another label discuss feedback_needed
Projects
None yet
Development

No branches or pull requests

4 participants