hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ramkrishna.s.vasudevan (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-15259) WALEdits under replay will also be replicated
Date Wed, 17 Feb 2016 05:21:18 GMT

    [ https://issues.apache.org/jira/browse/HBASE-15259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15149889#comment-15149889
] 

ramkrishna.s.vasudevan commented on HBASE-15259:
------------------------------------------------

LGTM.  May be you can also remove that code that sets the scope to null.

> WALEdits under replay will also be replicated
> ---------------------------------------------
>
>                 Key: HBASE-15259
>                 URL: https://issues.apache.org/jira/browse/HBASE-15259
>             Project: HBase
>          Issue Type: Bug
>            Reporter: ramkrishna.s.vasudevan
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Minor
>         Attachments: HBASE-15259.patch
>
>
> I need to verify this but seeing the code
> {code}
> try {
>         // We are about to append this edit; update the region-scoped sequence number.
 Do it
>         // here inside this single appending/writing thread.  Events are ordered on the
ringbuffer
>         // so region sequenceids will also be in order.
>         regionSequenceId = entry.stampRegionSequenceId();
>         // Edits are empty, there is nothing to append.  Maybe empty when we are looking
for a
>         // region sequence id only, a region edit/sequence id that is not associated
with an actual
>         // edit. It has to go through all the rigmarole to be sure we have the right
ordering.
>         if (entry.getEdit().isEmpty()) {
>           return;
>         }
>         // Coprocessor hook.
>         if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(),
>             entry.getEdit())) {
>           if (entry.getEdit().isReplay()) {
>             // Set replication scope null so that this won't be replicated
>             entry.getKey().setScopes(null);
>           }
>         }
>         if (!listeners.isEmpty()) {
>           for (WALActionsListener i: listeners) {
>             // TODO: Why does listener take a table description and CPs take a regioninfo?
 Fix.
>             i.visitLogEntryBeforeWrite(entry.getHTableDescriptor(), entry.getKey(),
>               entry.getEdit());
>           }
>         }
> {code}
> When a WALEdit is in replay we set the Logkey to null. But in the visitLogEntryBeforeWrite()
we again set the LogKey based on the replication scope associated with the cells. So the previous
step of setting null does not work here?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message