[
https://issues.apache.org/jira/browse/HBASE-13877?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14579697#comment-14579697
]
Enis Soztutar commented on HBASE-13877:
---------------------------------------
ITBLL Search job finds the missing keys in this WAL file:
{code}
2015-06-09 11:25:47,659 INFO [main] org.apache.hadoop.mapred.MapTask:
Processing split:
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/oldWALs/cn012.l42scl.hortonworks.com%2C16020%2C1433836445051.default.1433836858473
(-9223372036854775808:9223372036854775807) length:136606270
2015-06-09 11:25:47,678 INFO [main]
org.apache.hadoop.hbase.mapreduce.WALInputFormat: Opening reader for
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/oldWALs/cn012.l42scl.hortonworks.com%2C16020%2C1433836445051.default.1433836858473
(-9223372036854775808:9223372036854775807) length:136606270
2015-06-09 11:25:48,889 INFO [main]
org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList$Search: Loaded keys
to find: count=81027
2015-06-09 11:25:49,510 INFO [main]
org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList$Search: Found
cell=c\xEA\xC7\xDC\x16\xA8\x0A\x08\xE0\x18\xE8p\xF8\x0A)5/meta:prev/1433836859261/Put/vlen=16/seqid=0
2015-06-09 11:25:49,522 INFO [main]
org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList$Search: Found
cell=c\xEA\xC7\xDC\x16\xA8\x0A\x08\xE0\x18\xE8p\xF8\x0A)5/meta:count/1433836859261/Put/vlen=8/seqid=0
2015-06-09 11:25:49,523 INFO [main]
org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList$Search: Found
cell=c\xEA\xC7\xDC\x16\xA8\x0A\x08\xE0\x18\xE8p\xF8\x0A)5/meta:client/1433836859261/Put/vlen=71/seqid=0
2015-06-09 11:25:49,524 INFO [main]
org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList$Search: Found
cell=^\x04\xD2\x18T\x13\xA2\x83~N<\xD5\xA0\xECU\xEE/meta:prev/1433836859261/Put/vlen=16/seqid=0
2015-06-09 11:25:49,525 INFO [main]
org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList$Search: Found
cell=^\x04\xD2\x18T\x13\xA2\x83~N<\xD5\xA0\xECU\xEE/meta:count/1433836859261/Put/vlen=8/seqid=0
{code}
That particular WAL file have the missing keys at Sequence=82053:
{code}
First missing:
Sequence=82053 , region=acddcb7f27d64872b021900ad063f453 at write timestamp=Tue
Jun 09 01:00:59 PDT 2015
row=c\xEA\xC7\xDC\x16\xA8\x0A\x08\xE0\x18\xE8p\xF8\x0A)5, column=meta:prev
value: \x06\xA9)\xEF\x9D]~\x1C\x01b\xB4\x9E\x9D\x5Cv\x5C
row=c\xEA\xC7\xDC\x16\xA8\x0A\x08\xE0\x18\xE8p\xF8\x0A)5, column=meta:count
value: \x00\x00\x00\x00\x01<\x04y
row=c\xEA\xC7\xDC\x16\xA8\x0A\x08\xE0\x18\xE8p\xF8\x0A)5, column=meta:client
value: Job: job_1432937636181_0080 Task:
attempt_1432937636181_0080_m_000003_0
row=^\x04\xD2\x18T\x13\xA2\x83~N<\xD5\xA0\xECU\xEE, column=meta:prev
value: \xFD\xDE\xC9\xD6\xE4qU"\x8A\x92){\x15\xC7\x1F3
{code}
The same file also have non-missing keys with earlier seqIds for the same
region {{acddcb7f27d64872b021900ad063f453}}:
{code}
Sequence=81992 , region=acddcb7f27d64872b021900ad063f453 at write timestamp=Tue
Jun 09 01:00:58 PDT 2015
Sequence=81994 , region=acddcb7f27d64872b021900ad063f453 at write timestamp=Tue
Jun 09 01:00:58 PDT 2015
...
{code}
Last non missing:
{code}
Sequence=82042 , region=acddcb7f27d64872b021900ad063f453 at write timestamp=Tue
Jun 09 01:00:59 PDT 2015
row=Y\x99\x99\x99\x99\x99\x99\x94, column=METAFAMILY:HBASE::FLUSH
value: \x08\x00\x12\x1CIntegrationTestBigLinkedList\x1A
acddcb7f27d64872b021900ad063f453
\xF9\x80\x05*\x0E\x0A\x05ErkuW\x12\x05ErkuW*\x0A\x0A\x03big\x12\x03big*\x0E\x0A\x05fcSFt\x12\x05fcSFt*\x0C\x0
A\x04meta\x12\x04meta*\x0C\x0A\x04tiny\x12\x04tiny2UIntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
{code}
As you can see, for this region {{acddcb7f27d64872b021900ad063f453}}, we have
the keys with seqId up to 82042, but we miss some of the keys with seqid
82053+.
Looking at the WAL splitting, it seems that everything below seqId 82627 is
skipped:
Master:
{code}
2015-06-09 01:02:13,296 INFO [RS_LOG_REPLAY_OPS-cn015:16020-1]
wal.WALSplitter: Splitting wal:
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/WALs/cn012.l42scl.hortonworks.com,16020,1433836445051-splitting/cn012.l42scl.hortonworks.com%2C16020%2C1433836445051.default.1433836858473,
length=136606270
2015-06-09 01:02:14,103 INFO [RS_LOG_REPLAY_OPS-cn015:16020-1]
wal.WALSplitter: Processed 247 edits across 3 regions; edits skipped=3210; log
file=hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/WALs/cn012.l42scl.hortonworks.com,16020,1433836445051-splitting/cn012.l42scl.hortonworks.com%2C16020%2C1433836445051.default.1433836858473,
length=136606270, corrupted=false, progress failed=false
{code}
Earliest recovered.edits:
{code}
2015-06-09 01:02:12,152 DEBUG [split-log-closeStream-2] wal.WALSplitter: Closed
wap
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/recovered.edits/0000000000000082627.temp
(wrote 12 edits, skipped 0 edits in 17ms
2015-06-09 01:02:12,332 INFO [split-log-closeStream-2] wal.WALSplitter: Rename
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/recovered.edits/0000000000000082627.temp
to
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/recovered.edits/0000000000000082650
{code}
This filtering is the correct behavior since it seems that we have done a flush
with seqid=82626 earlier:
{code}
2015-06-09 01:01:13,274 INFO
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool7-thread-1]
regionserver.DefaultStoreFlusher: Flushed, sequenceid=82626, memsize=280.7 M,
hasBloomFilter=false, into tmp file
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/.tmp/26d0cfddc2a242f396447e51629bf74e
{code}
However, looking at the contents of this hfile
{{26d0cfddc2a242f396447e51629bf74e}}, we can find only cells with seqIds in the
range of 78284 to 82040.
{code}
hbase hfile -p -f
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/archive/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/meta/26d0cfddc2a242f396447e51629bf74e
| grep "seqid=" | cut -d "/" -f 6 | cut -f 2 -d "=" | cut -f 1 -d " " | sort
| less
{code}
However, the hfile metadata says that the file has a maxSeqId of 82626:
{code}
hbase hfile -m -f
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/archive/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/meta/26d0cfddc2a242f396447e51629bf74e
...
Fileinfo:
DATA_BLOCK_ENCODING = DIFF
DELETE_FAMILY_COUNT = \x00\x00\x00\x00\x00\x00\x00\x00
EARLIEST_PUT_TS = \x00\x00\x01M\xD7UG\xFF
KEY_VALUE_VERSION = \x00\x00\x00\x01
MAJOR_COMPACTION_KEY = \x00
MAX_MEMSTORE_TS_KEY = \x00\x00\x00\x00\x00\x01@x
MAX_SEQ_ID_KEY = 82626
...
{code}
As you can see, the missing keys range from 82053 - 82626, so the difference in
this hfiles contents and missing keys fits perfectly.
The flush writing this hfile with seqId=82626 complains about {{Snapshot called
again without clearing previous}}:
{code}
2015-06-09 01:00:59,552 DEBUG [B.defaultRpcServer.handler=2,queue=2,port=16020]
regionserver.HRegion: Flush requested on
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
2015-06-09 01:01:06,776 DEBUG
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool7-thread-1]
flush.FlushTableSubprocedure: Flush region
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
started...
2015-06-09 01:01:06,776 INFO
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool7-thread-1]
regionserver.HRegion: Started memstore flush for
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.,
current region memstore size 328.90 MB, and 5/5 column families' memstores are
being flushed.
2015-06-09 01:01:13,274 INFO
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool7-thread-1]
regionserver.DefaultStoreFlusher: Flushed, sequenceid=82626, memsize=280.7 M,
hasBloomFilter=false, into tmp file
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/.tmp/26d0cfddc2a242f396447e51629bf74e
2015-06-09 01:01:06,787 WARN
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool7-thread-1]
regionserver.DefaultMemStore: Snapshot called again without clearing previous.
Doing nothing. Another ongoing flush or did we fail last attempt?
2015-06-09 01:01:13,294 INFO
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool7-thread-1]
regionserver.HStore: Added
hdfs://cn011.l42scl.hortonworks.com:8020/apps/hbase/data/data/default/IntegrationTestBigLinkedList/acddcb7f27d64872b021900ad063f453/meta/26d0cfddc2a242f396447e51629bf74e,
entries=1414983, sequenceid=82626, filesize=69.4 M
{code}
Now, coming to the fun part. Going back to earlier in the RS log, an earlier
flush attempt from {{FlushTableSubprocedure}} was cancelled and it left the
memstore snapshot not being cleared:
{code}
2015-06-09 01:00:28,239 DEBUG [B.defaultRpcServer.handler=4,queue=4,port=16020]
regionserver.HRegion: Flush requested on
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
2015-06-09 01:00:28,239 WARN [MemStoreFlusher.1] regionserver.MemStoreFlusher:
Region
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
has too many store files; delaying flush up to 90000ms
2015-06-09 01:00:59,197 DEBUG
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool6-thread-1]
flush.FlushTableSubprocedure: Starting region operation on
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
2015-06-09 01:00:59,198 DEBUG
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool6-thread-1]
flush.FlushTableSubprocedure: Flush region
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.
started...
2015-06-09 01:00:59,199 INFO
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool6-thread-1]
regionserver.HRegion: Started memstore flush for
IntegrationTestBigLinkedList,Y\x99\x99\x99\x99\x99\x99\x94,1433835414361.acddcb7f27d64872b021900ad063f453.,
current region memstore size 280.68 MB, and 5/5 column families' memstores are
being flushed.
2015-06-09 01:00:59,520 WARN
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool6-thread-1]
wal.FSHLog: Interrupted
java.lang.InterruptedException
at java.lang.Object.wait(Native Method)
at
org.apache.hadoop.hbase.regionserver.wal.SyncFuture.get(SyncFuture.java:167)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.blockOnSync(FSHLog.java:1472)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.publishSyncThenBlockOnCompletion(FSHLog.java:1466)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1585)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalPrepareFlushCache(HRegion.java:2205)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2029)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1994)
at
org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1886)
at org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1813)
at
org.apache.hadoop.hbase.procedure.flush.FlushTableSubprocedure$RegionFlushTask.call(FlushTableSubprocedure.java:69)
at
org.apache.hadoop.hbase.procedure.flush.FlushTableSubprocedure$RegionFlushTask.call(FlushTableSubprocedure.java:57)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
2015-06-09 01:00:59,521 WARN
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool6-thread-1]
regionserver.HRegion: Unexpected exception while wal.sync(), ignoring.
Exception: java.io.InterruptedIOException
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.convertInterruptedExceptionToIOException(FSHLog.java:1484)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.blockOnSync(FSHLog.java:1476)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.publishSyncThenBlockOnCompletion(FSHLog.java:1466)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1585)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalPrepareFlushCache(HRegion.java:2205)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2029)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1994)
at
org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1886)
at org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1813)
at
org.apache.hadoop.hbase.procedure.flush.FlushTableSubprocedure$RegionFlushTask.call(FlushTableSubprocedure.java:69)
at
org.apache.hadoop.hbase.procedure.flush.FlushTableSubprocedure$RegionFlushTask.call(FlushTableSubprocedure.java:57)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.InterruptedException
at java.lang.Object.wait(Native Method)
at
org.apache.hadoop.hbase.regionserver.wal.SyncFuture.get(SyncFuture.java:167)
at
org.apache.hadoop.hbase.regionserver.wal.FSHLog.blockOnSync(FSHLog.java:1472)
... 15 more
2015-06-09 01:00:59,522 WARN
[rs(cn012.l42scl.hortonworks.com,16020,1433836445051)-flush-proc-pool6-thread-1]
ipc.Client: interrupted waiting to send rpc request to server
java.lang.InterruptedException
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:400)
at java.util.concurrent.FutureTask.get(FutureTask.java:187)
at
org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1046)
at org.apache.hadoop.ipc.Client.call(Client.java:1441)
at org.apache.hadoop.ipc.Client.call(Client.java:1399)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
at com.sun.proxy.$Proxy19.getFileInfo(Unknown Source)
at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:752)
at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
at com.sun.proxy.$Proxy20.getFileInfo(Unknown Source)
at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at org.apache.hadoop.hbase.fs.HFileSystem$1.invoke(HFileSystem.java:279)
at com.sun.proxy.$Proxy21.getFileInfo(Unknown Source)
at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1988)
at
org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1118)
at
org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1114)
at
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1114)
at
org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:409)
at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1400)
at
org.apache.hadoop.hbase.regionserver.StoreFile$WriterBuilder.build(StoreFile.java:629)
at
org.apache.hadoop.hbase.regionserver.HStore.createWriterInTmp(HStore.java:1007)
at
org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher.flushSnapshot(DefaultStoreFlusher.java:66)
at
org.apache.hadoop.hbase.regionserver.HStore.flushCache(HStore.java:920)
at
org.apache.hadoop.hbase.regionserver.HStore$StoreFlusherImpl.flushCache(HStore.java:2192)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushCacheAndCommit(HRegion.java:2282)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2032)
at
org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1994)
at
org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1886)
at org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1813)
at
org.apache.hadoop.hbase.procedure.flush.FlushTableSubprocedure$RegionFlushTask.call(FlushTableSubprocedure.java:69)
at
org.apache.hadoop.hbase.procedure.flush.FlushTableSubprocedure$RegionFlushTask.call(FlushTableSubprocedure.java:57)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}
So it seems that the interrupt from {{FlushTableSubprocedure}} causes the flush
to abort after the prepare step. This leaves the memstore snapshot to be left
there. A later attempt to flush, then uses the same snapshot (see {{Snapshot
called again without clearing previous}} above), but obtains the next seqId
from the region:
{code}
flushOpSeqId = getNextSequenceId(wal);
{code}
This flushes entries in the non-cleared snapshot with seqIds in range
78284-82040, but does not persist entries with seqId 82053 - 82626. However,
since the flushOpSeqId is obtained from regions next seqId, the entries are
written with 82626.
> Interrupt to flush from TableFlushProcedure causes dataloss in ITBLL
> --------------------------------------------------------------------
>
> Key: HBASE-13877
> URL: https://issues.apache.org/jira/browse/HBASE-13877
> Project: HBase
> Issue Type: Bug
> Reporter: Enis Soztutar
> Assignee: Enis Soztutar
> Priority: Blocker
> Fix For: 2.0.0, 1.2.0, 1.1.1
>
>
> ITBLL with 1.25B rows failed for me (and Stack as reported in
> https://issues.apache.org/jira/browse/HBASE-13811?focusedCommentId=14577834&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14577834)
>
> HBASE-13811 and HBASE-13853 fixed an issue with WAL edit filtering.
> The root cause this time seems to be different. It is due to procedure based
> flush interrupting the flush request in case the procedure is cancelled from
> an exception elsewhere. This leaves the memstore snapshot intact without
> aborting the server. The next flush, then flushes the previous memstore with
> the current seqId (as opposed to seqId from the memstore snapshot). This
> creates an hfile with larger seqId than what its contents are. Previous
> behavior in 0.98 and 1.0 (I believe) is that after flush prepare and
> interruption / exception will cause RS abort.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)