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.HBaseTestingUtility; 035import org.apache.hadoop.hbase.client.AsyncAdmin; 036import org.apache.hadoop.hbase.client.AsyncConnection; 037import org.apache.hadoop.hbase.client.ConnectionFactory; 038import org.apache.hadoop.hbase.replication.DummyReplicationEndpoint; 039import org.apache.hadoop.hbase.replication.FSReplicationPeerStorage; 040import org.apache.hadoop.hbase.replication.ReplicationException; 041import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 042import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; 043import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; 044import org.apache.hadoop.hbase.testclassification.LargeTests; 045import org.apache.hadoop.hbase.testclassification.MasterTests; 046import org.junit.AfterClass; 047import org.junit.Before; 048import org.junit.BeforeClass; 049import org.junit.ClassRule; 050import org.junit.Test; 051import org.junit.experimental.categories.Category; 052import org.junit.runner.RunWith; 053import org.junit.runners.Parameterized; 054import org.junit.runners.Parameterized.Parameter; 055import org.junit.runners.Parameterized.Parameters; 056 057import org.apache.hbase.thirdparty.com.google.common.io.Closeables; 058 059@RunWith(Parameterized.class) 060@Category({ MasterTests.class, LargeTests.class }) 061public class TestDisablePeerModification { 062 063 @ClassRule 064 public static final HBaseClassTestRule CLASS_RULE = 065 HBaseClassTestRule.forClass(TestDisablePeerModification.class); 066 067 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); 068 069 private static volatile CountDownLatch ARRIVE; 070 071 private static volatile CountDownLatch RESUME; 072 073 public static final class MockPeerStorage extends FSReplicationPeerStorage { 074 075 public MockPeerStorage(FileSystem fs, Configuration conf) throws IOException { 076 super(fs, conf); 077 } 078 079 @Override 080 public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) 081 throws ReplicationException { 082 ARRIVE.countDown(); 083 try { 084 RESUME.await(); 085 } catch (InterruptedException e) { 086 throw new ReplicationException(e); 087 } 088 super.addPeer(peerId, peerConfig, enabled); 089 } 090 } 091 092 private static AsyncConnection CONN; 093 094 @Parameter 095 public boolean async; 096 097 @Parameters(name = "{index}: async={0}") 098 public static List<Object[]> params() { 099 return Arrays.asList(new Object[] { true }, new Object[] { false }); 100 } 101 102 @BeforeClass 103 public static void setUp() throws Exception { 104 UTIL.getConfiguration().setClass(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, 105 MockPeerStorage.class, ReplicationPeerStorage.class); 106 UTIL.startMiniCluster(1); 107 CONN = ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get(); 108 } 109 110 @AfterClass 111 public static void tearDown() throws IOException { 112 Closeables.close(CONN, true); 113 UTIL.shutdownMiniCluster(); 114 } 115 116 @Before 117 public void setUpBeforeTest() throws IOException { 118 UTIL.getAdmin().replicationPeerModificationSwitch(true, true); 119 } 120 121 @Test 122 public void testDrainProcs() throws Exception { 123 ARRIVE = new CountDownLatch(1); 124 RESUME = new CountDownLatch(1); 125 AsyncAdmin admin = CONN.getAdmin(); 126 ReplicationPeerConfig rpc = 127 ReplicationPeerConfig.newBuilder().setClusterKey(UTIL.getClusterKey() + "-test") 128 .setReplicationEndpointImpl(DummyReplicationEndpoint.class.getName()).build(); 129 CompletableFuture<Void> addFuture = admin.addReplicationPeer("test_peer_" + async, rpc); 130 ARRIVE.await(); 131 132 // we have a pending add peer procedure which has already passed the first state, let's issue a 133 // peer modification switch request to disable peer modification and set drainProcs to true 134 CompletableFuture<Boolean> switchFuture; 135 if (async) { 136 switchFuture = admin.replicationPeerModificationSwitch(false, true); 137 } else { 138 switchFuture = new CompletableFuture<>(); 139 ForkJoinPool.commonPool().submit(() -> { 140 try { 141 switchFuture.complete(UTIL.getAdmin().replicationPeerModificationSwitch(false, true)); 142 } catch (IOException e) { 143 switchFuture.completeExceptionally(e); 144 } 145 }); 146 } 147 148 // sleep a while, the switchFuture should not finish yet 149 // the sleep is necessary as we can not join on the switchFuture, so there is no stable way to 150 // make sure we have already changed the flag at master side, sleep a while is the most suitable 151 // way here 152 Thread.sleep(5000); 153 assertFalse(switchFuture.isDone()); 154 155 // also verify that we can not schedule a new peer modification procedure 156 AddPeerProcedure proc = new AddPeerProcedure("failure", rpc, true); 157 UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().submitProcedure(proc); 158 UTIL.waitFor(15000, () -> proc.isFinished()); 159 // make sure the procedure is failed because of peer modification disabled 160 assertTrue(proc.isFailed()); 161 assertThat(proc.getException().getCause().getMessage(), 162 containsString("Replication peer modification disabled")); 163 164 // sleep a while and check again, make sure the switchFuture is still not done 165 Thread.sleep(5000); 166 assertFalse(switchFuture.isDone()); 167 168 // resume the add peer procedure and wait it done 169 RESUME.countDown(); 170 addFuture.get(); 171 172 // this time the switchFuture should be able to finish 173 assertTrue(switchFuture.get()); 174 } 175}