001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master.replication; 019 020import static org.hamcrest.MatcherAssert.assertThat; 021import static org.hamcrest.Matchers.containsString; 022import static org.junit.Assert.assertFalse; 023import static org.junit.Assert.assertTrue; 024 025import java.io.IOException; 026import java.util.Arrays; 027import java.util.List; 028import java.util.concurrent.CompletableFuture; 029import java.util.concurrent.CountDownLatch; 030import java.util.concurrent.ForkJoinPool; 031import org.apache.hadoop.conf.Configuration; 032import org.apache.hadoop.fs.FileSystem; 033import org.apache.hadoop.hbase.HBaseClassTestRule; 034import org.apache.hadoop.hbase.HBaseTestingUtil; 035import org.apache.hadoop.hbase.client.AsyncAdmin; 036import org.apache.hadoop.hbase.replication.DummyReplicationEndpoint; 037import org.apache.hadoop.hbase.replication.FSReplicationPeerStorage; 038import org.apache.hadoop.hbase.replication.ReplicationException; 039import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 040import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; 041import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; 042import org.apache.hadoop.hbase.replication.SyncReplicationState; 043import org.apache.hadoop.hbase.testclassification.LargeTests; 044import org.apache.hadoop.hbase.testclassification.MasterTests; 045import org.junit.AfterClass; 046import org.junit.Before; 047import org.junit.BeforeClass; 048import org.junit.ClassRule; 049import org.junit.Test; 050import org.junit.experimental.categories.Category; 051import org.junit.runner.RunWith; 052import org.junit.runners.Parameterized; 053import org.junit.runners.Parameterized.Parameter; 054import org.junit.runners.Parameterized.Parameters; 055 056@RunWith(Parameterized.class) 057@Category({ MasterTests.class, LargeTests.class }) 058public class TestDisablePeerModification { 059 060 @ClassRule 061 public static final HBaseClassTestRule CLASS_RULE = 062 HBaseClassTestRule.forClass(TestDisablePeerModification.class); 063 064 private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); 065 066 private static volatile CountDownLatch ARRIVE; 067 068 private static volatile CountDownLatch RESUME; 069 070 public static final class MockPeerStorage extends FSReplicationPeerStorage { 071 072 public MockPeerStorage(FileSystem fs, Configuration conf) throws IOException { 073 super(fs, conf); 074 } 075 076 @Override 077 public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled, 078 SyncReplicationState syncReplicationState) throws ReplicationException { 079 ARRIVE.countDown(); 080 try { 081 RESUME.await(); 082 } catch (InterruptedException e) { 083 throw new ReplicationException(e); 084 } 085 super.addPeer(peerId, peerConfig, enabled, syncReplicationState); 086 } 087 } 088 089 @Parameter 090 public boolean async; 091 092 @Parameters(name = "{index}: async={0}") 093 public static List<Object[]> params() { 094 return Arrays.asList(new Object[] { true }, new Object[] { false }); 095 } 096 097 @BeforeClass 098 public static void setUp() throws Exception { 099 UTIL.getConfiguration().setClass(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, 100 MockPeerStorage.class, ReplicationPeerStorage.class); 101 UTIL.startMiniCluster(1); 102 } 103 104 @AfterClass 105 public static void tearDown() throws IOException { 106 UTIL.shutdownMiniCluster(); 107 } 108 109 @Before 110 public void setUpBeforeTest() throws IOException { 111 UTIL.getAdmin().replicationPeerModificationSwitch(true, true); 112 } 113 114 @Test 115 public void testDrainProcs() throws Exception { 116 ARRIVE = new CountDownLatch(1); 117 RESUME = new CountDownLatch(1); 118 AsyncAdmin admin = UTIL.getAsyncConnection().getAdmin(); 119 ReplicationPeerConfig rpc = 120 ReplicationPeerConfig.newBuilder().setClusterKey(UTIL.getRpcConnnectionURI() + "-test") 121 .setReplicationEndpointImpl(DummyReplicationEndpoint.class.getName()).build(); 122 CompletableFuture<Void> addFuture = admin.addReplicationPeer("test_peer_" + async, rpc); 123 ARRIVE.await(); 124 125 // we have a pending add peer procedure which has already passed the first state, let's issue a 126 // peer modification switch request to disable peer modification and set drainProcs to true 127 CompletableFuture<Boolean> switchFuture; 128 if (async) { 129 switchFuture = admin.replicationPeerModificationSwitch(false, true); 130 } else { 131 switchFuture = new CompletableFuture<>(); 132 ForkJoinPool.commonPool().submit(() -> { 133 try { 134 switchFuture.complete(UTIL.getAdmin().replicationPeerModificationSwitch(false, true)); 135 } catch (IOException e) { 136 switchFuture.completeExceptionally(e); 137 } 138 }); 139 } 140 141 // sleep a while, the switchFuture should not finish yet 142 // the sleep is necessary as we can not join on the switchFuture, so there is no stable way to 143 // make sure we have already changed the flag at master side, sleep a while is the most suitable 144 // way here 145 Thread.sleep(5000); 146 assertFalse(switchFuture.isDone()); 147 148 // also verify that we can not schedule a new peer modification procedure 149 AddPeerProcedure proc = new AddPeerProcedure("failure", rpc, true); 150 UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().submitProcedure(proc); 151 UTIL.waitFor(15000, () -> proc.isFinished()); 152 // make sure the procedure is failed because of peer modification disabled 153 assertTrue(proc.isFailed()); 154 assertThat(proc.getException().getCause().getMessage(), 155 containsString("Replication peer modification disabled")); 156 157 // sleep a while and check again, make sure the switchFuture is still not done 158 Thread.sleep(5000); 159 assertFalse(switchFuture.isDone()); 160 161 // resume the add peer procedure and wait it done 162 RESUME.countDown(); 163 addFuture.get(); 164 165 // this time the switchFuture should be able to finish 166 assertTrue(switchFuture.get()); 167 } 168}