-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-29796 Allow sleepForRetry replication config to be overridden by replication peers #7577
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
Changes from 15 commits
f405fb8
d97bf18
ff20206
207dacd
e40230a
0487429
251d4f0
cae2efb
49f6d52
03eaa64
92f68af
de0064e
7e4afc3
db8fa8c
72e9655
851c737
d75920c
13b7661
2d412fe
46b1533
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -862,4 +862,12 @@ public String logPeerId() { | |
| public long getTotalReplicatedEdits() { | ||
| return totalReplicatedEdits.get(); | ||
| } | ||
|
|
||
| public long getSleepForRetries() { | ||
|
||
| return sleepForRetries; | ||
| } | ||
|
|
||
| public Map<String, ReplicationSourceShipper> getWorkerThreadsForTesting() { | ||
|
||
| return workerThreads; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -355,4 +355,12 @@ void clearWALEntryBatch() { | |
| totalReleasedBytes); | ||
| } | ||
| } | ||
|
|
||
| long getSleepForRetries() { | ||
| return sleepForRetries; | ||
| } | ||
|
|
||
| ReplicationSourceWALReader getEntryReaderForTesting() { | ||
|
||
| return entryReader; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,135 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.hadoop.hbase.replication.regionserver; | ||
|
|
||
| import static org.mockito.ArgumentMatchers.anyString; | ||
| import static org.mockito.Mockito.mock; | ||
| import static org.mockito.Mockito.never; | ||
| import static org.mockito.Mockito.times; | ||
| import static org.mockito.Mockito.verify; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| import org.apache.hadoop.hbase.HBaseClassTestRule; | ||
| import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; | ||
| import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; | ||
| import org.apache.hadoop.hbase.replication.ReplicationPeerImpl; | ||
| import org.apache.hadoop.hbase.replication.ReplicationPeers; | ||
| import org.apache.hadoop.hbase.testclassification.ReplicationTests; | ||
| import org.apache.hadoop.hbase.testclassification.SmallTests; | ||
| import org.junit.Before; | ||
| import org.junit.ClassRule; | ||
| import org.junit.Test; | ||
| import org.junit.experimental.categories.Category; | ||
|
|
||
| @Category({ ReplicationTests.class, SmallTests.class }) | ||
|
||
| public class TestPeerProcedureHandlerImpl { | ||
|
|
||
| @ClassRule | ||
| public static final HBaseClassTestRule CLASS_RULE = | ||
| HBaseClassTestRule.forClass(TestPeerProcedureHandlerImpl.class); | ||
|
|
||
| private ReplicationSourceManager mockSourceManager; | ||
| private ReplicationPeers mockReplicationPeers; | ||
| private ReplicationPeerImpl mockPeer; | ||
| private PeerProcedureHandlerImpl handler; | ||
| private static final String PEER_ID = "testPeer"; | ||
|
|
||
| @Before | ||
| public void setup() throws Exception { | ||
| mockSourceManager = mock(ReplicationSourceManager.class); | ||
| mockReplicationPeers = mock(ReplicationPeers.class); | ||
| mockPeer = mock(ReplicationPeerImpl.class); | ||
|
|
||
| when(mockSourceManager.getReplicationPeers()).thenReturn(mockReplicationPeers); | ||
| when(mockReplicationPeers.getPeer(PEER_ID)).thenReturn(mockPeer); | ||
|
|
||
| handler = new PeerProcedureHandlerImpl(mockSourceManager, null); | ||
| } | ||
|
|
||
| @Test | ||
| public void testReplicationSourceConfigChangeTriggers() throws Exception { | ||
| ReplicationPeerConfig oldConfig = ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster") | ||
| .putConfiguration("replication.source.sleepforretries", "1000").build(); | ||
|
|
||
| ReplicationPeerConfig newConfig = ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster") | ||
| .putConfiguration("replication.source.sleepforretries", "5000").build(); | ||
|
|
||
| when(mockPeer.getPeerConfig()).thenReturn(oldConfig); | ||
| when(mockPeer.getPeerState()).thenReturn(PeerState.ENABLED); | ||
| when(mockReplicationPeers.refreshPeerConfig(PEER_ID)).thenReturn(newConfig); | ||
| when(mockReplicationPeers.refreshPeerState(PEER_ID)).thenReturn(PeerState.ENABLED); | ||
|
|
||
| handler.updatePeerConfig(PEER_ID); | ||
|
|
||
| verify(mockSourceManager, times(1)).refreshSources(PEER_ID); | ||
| } | ||
|
|
||
| @Test | ||
| public void testNonReplicationSourceConfigDoesNotTrigger() throws Exception { | ||
| ReplicationPeerConfig oldConfig = ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster") | ||
| .putConfiguration("some.other.config", "value1").build(); | ||
|
|
||
| ReplicationPeerConfig newConfig = ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster") | ||
| .putConfiguration("some.other.config", "value2").build(); | ||
|
|
||
| when(mockPeer.getPeerConfig()).thenReturn(oldConfig); | ||
| when(mockPeer.getPeerState()).thenReturn(PeerState.ENABLED); | ||
| when(mockReplicationPeers.refreshPeerConfig(PEER_ID)).thenReturn(newConfig); | ||
| when(mockReplicationPeers.refreshPeerState(PEER_ID)).thenReturn(PeerState.ENABLED); | ||
|
|
||
| handler.updatePeerConfig(PEER_ID); | ||
|
|
||
| verify(mockSourceManager, never()).refreshSources(anyString()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testNewReplicationSourceConfigTriggers() throws Exception { | ||
| ReplicationPeerConfig oldConfig = | ||
| ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster").build(); | ||
|
|
||
| ReplicationPeerConfig newConfig = ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster") | ||
| .putConfiguration("replication.source.sleepforretries", "5000").build(); | ||
|
|
||
| when(mockPeer.getPeerConfig()).thenReturn(oldConfig); | ||
| when(mockPeer.getPeerState()).thenReturn(PeerState.ENABLED); | ||
| when(mockReplicationPeers.refreshPeerConfig(PEER_ID)).thenReturn(newConfig); | ||
| when(mockReplicationPeers.refreshPeerState(PEER_ID)).thenReturn(PeerState.ENABLED); | ||
|
|
||
| handler.updatePeerConfig(PEER_ID); | ||
|
|
||
| verify(mockSourceManager, times(1)).refreshSources(PEER_ID); | ||
| } | ||
|
|
||
| @Test | ||
| public void testRemovedReplicationSourceConfigTriggers() throws Exception { | ||
| ReplicationPeerConfig oldConfig = ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster") | ||
| .putConfiguration("replication.source.sleepforretries", "2000").build(); | ||
|
|
||
| ReplicationPeerConfig newConfig = | ||
| ReplicationPeerConfig.newBuilder().setClusterKey("oldCluster").build(); | ||
|
|
||
| when(mockPeer.getPeerConfig()).thenReturn(oldConfig); | ||
| when(mockPeer.getPeerState()).thenReturn(PeerState.ENABLED); | ||
| when(mockReplicationPeers.refreshPeerConfig(PEER_ID)).thenReturn(newConfig); | ||
| when(mockReplicationPeers.refreshPeerState(PEER_ID)).thenReturn(PeerState.ENABLED); | ||
|
|
||
| handler.updatePeerConfig(PEER_ID); | ||
|
|
||
| verify(mockSourceManager, times(1)).refreshSources(PEER_ID); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do we need to handle the following keys? if not, then your change looks good to me
hbase.replication.source.fs.conf.provider
hbase.replication.source.service
hbase.replication.source.maxthreads
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As far as I can tell, the first two couldn't be used on a per-peer basis while the last is only used by the InterClusterReplicationEndpoint and not a ReplicationSource. Therefore, I think it is fine to not include those.