You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Tomu Tsuruhara (JIRA)" <ji...@apache.org> on 2017/04/05 22:36:42 UTC
[jira] [Comment Edited] (HBASE-17871) scan#setBatch(int) call leads
wrong result of VerifyReplication
[ https://issues.apache.org/jira/browse/HBASE-17871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15957920#comment-15957920 ]
Tomu Tsuruhara edited comment on HBASE-17871 at 4/5/17 10:36 PM:
-----------------------------------------------------------------
Thanks! Addressed findbugs warning in v3 patch.
"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.
> Have you verified the change fixes the inconsistency ?
Yes, I confirmed that in my environment.
before the patch:
!beforethepatch.png|thumbnail!
after:
!after.png|thumbnail!
was (Author: tomu.tsuruhara):
@Ted Yu Thanks! Addressed findbugs warning in v3 patch.
"--batch" parsing code was introduced in HBASE-15191.
We don't need to put extra code this time.
> Have you verified the change fixes the inconsistency ?
Yes, I confirmed that in my environment.
before the patch:
!スクリーンショット 2017-04-06 7.19.53.png|thumbnail=1!
after:
!スクリーンショット 2017-04-06 7.21.53.png|thumbnail=1!
> scan#setBatch(int) call leads wrong result of VerifyReplication
> ---------------------------------------------------------------
>
> Key: HBASE-17871
> URL: https://issues.apache.org/jira/browse/HBASE-17871
> Project: HBase
> Issue Type: Bug
> Affects Versions: 2.0.0, 1.4.0
> Reporter: Tomu Tsuruhara
> Assignee: Tomu Tsuruhara
> Priority: Minor
> Attachments: after.png, beforethepatch.png, HBASE-17871.master.001.patch, HBASE-17871.master.002.patch, HBASE-17871.master.003.patch
>
>
> VerifyReplication tool printed weird logs.
> {noformat}
> 2017-04-03 23:30:50,252 ERROR [main] org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: CONTENT_DIFFERENT_ROWS, rowkey=a00001001930000
> 2017-04-03 23:30:50,280 ERROR [main] org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: ONLY_IN_PEER_TABLE_ROWS, rowkey=a00001001930000
> 2017-04-03 23:30:50,387 ERROR [main] org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: CONTENT_DIFFERENT_ROWS, rowkey=a00001003850000
> 2017-04-03 23:30:50,414 ERROR [main] org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: ONLY_IN_PEER_TABLE_ROWS, rowkey=a00001003850000
> 2017-04-03 23:30:50,480 ERROR [main] org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: CONTENT_DIFFERENT_ROWS, rowkey=a00001005320000
> 2017-04-03 23:30:50,508 ERROR [main] org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication: ONLY_IN_PEER_TABLE_ROWS, rowkey=a00001005320000
> {noformat}
> Here, each bad rows were marked as both {{CONTENT_DIFFERENT_ROWS}} and {{ONLY_IN_PEER_TABLE_ROWS}}.
> This should never happen so I took a look at code and found scan.setBatch call.
> {code}
> @Override
> public void map(ImmutableBytesWritable row, final Result value,
> Context context)
> throws IOException {
> if (replicatedScanner == null) {
> ...
> final Scan scan = new Scan();
> scan.setBatch(batch);
> {code}
> As stated in HBASE-16376, {{scan#setBatch(int)}} call implicitly allows scan results to be partial.
> Since {{VerifyReplication}} is assuming each {{scanner.next()}} call returns entire row,
> partial results break compare logic.
> We should avoid setBatch call here.
> Thanks to RPC chunking (explained in this blog https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1),
> it's safe and acceptable I think.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)