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.regionserver; 019 020import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID; 021import static org.apache.hadoop.hbase.HConstants.REPLICATION_CONF_DIR; 022import static org.junit.Assert.assertEquals; 023import static org.junit.Assert.assertTrue; 024 025import java.io.File; 026import java.io.FileOutputStream; 027import java.io.IOException; 028import java.util.List; 029import java.util.Map; 030import java.util.Optional; 031import java.util.concurrent.CountDownLatch; 032import java.util.concurrent.TimeUnit; 033import java.util.concurrent.atomic.AtomicInteger; 034import org.apache.hadoop.conf.Configuration; 035import org.apache.hadoop.fs.FSDataOutputStream; 036import org.apache.hadoop.fs.Path; 037import org.apache.hadoop.hbase.Cell; 038import org.apache.hadoop.hbase.CellBuilder; 039import org.apache.hadoop.hbase.CellBuilderFactory; 040import org.apache.hadoop.hbase.CellBuilderType; 041import org.apache.hadoop.hbase.HBaseClassTestRule; 042import org.apache.hadoop.hbase.HBaseTestingUtil; 043import org.apache.hadoop.hbase.HConstants; 044import org.apache.hadoop.hbase.KeyValue; 045import org.apache.hadoop.hbase.TableName; 046import org.apache.hadoop.hbase.client.Admin; 047import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 048import org.apache.hadoop.hbase.client.Connection; 049import org.apache.hadoop.hbase.client.ConnectionFactory; 050import org.apache.hadoop.hbase.client.Get; 051import org.apache.hadoop.hbase.client.Result; 052import org.apache.hadoop.hbase.client.Table; 053import org.apache.hadoop.hbase.client.TableDescriptor; 054import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 055import org.apache.hadoop.hbase.coprocessor.ObserverContext; 056import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; 057import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; 058import org.apache.hadoop.hbase.coprocessor.RegionObserver; 059import org.apache.hadoop.hbase.io.hfile.HFile; 060import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; 061import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 062import org.apache.hadoop.hbase.replication.TestReplicationBase; 063import org.apache.hadoop.hbase.testclassification.MediumTests; 064import org.apache.hadoop.hbase.testclassification.ReplicationTests; 065import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; 066import org.apache.hadoop.hbase.util.Bytes; 067import org.apache.hadoop.hbase.util.Pair; 068import org.apache.hadoop.hdfs.MiniDFSCluster; 069import org.junit.After; 070import org.junit.Before; 071import org.junit.BeforeClass; 072import org.junit.ClassRule; 073import org.junit.Rule; 074import org.junit.Test; 075import org.junit.experimental.categories.Category; 076import org.junit.rules.TemporaryFolder; 077import org.junit.rules.TestName; 078import org.slf4j.Logger; 079import org.slf4j.LoggerFactory; 080 081/** 082 * Integration test for bulk load replication. Defines three clusters, with the following 083 * replication topology: "1 <-> 2 <-> 3" (active-active between 1 and 2, and active-active between 2 084 * and 3). For each of defined test clusters, it performs a bulk load, asserting values on bulk 085 * loaded file gets replicated to other two peers. Since we are doing 3 bulk loads, with the given 086 * replication topology all these bulk loads should get replicated only once on each peer. To assert 087 * this, this test defines a preBulkLoad coprocessor and adds it to all test table regions, on each 088 * of the clusters. This CP counts the amount of times bulk load actually gets invoked, certifying 089 * we are not entering the infinite loop condition addressed by HBASE-22380. 090 */ 091@Category({ ReplicationTests.class, MediumTests.class }) 092public class TestBulkLoadReplication extends TestReplicationBase { 093 094 @ClassRule 095 public static final HBaseClassTestRule CLASS_RULE = 096 HBaseClassTestRule.forClass(TestBulkLoadReplication.class); 097 098 protected static final Logger LOG = LoggerFactory.getLogger(TestBulkLoadReplication.class); 099 100 private static final String PEER1_CLUSTER_ID = "peer1"; 101 private static final String PEER2_CLUSTER_ID = "peer2"; 102 private static final String PEER3_CLUSTER_ID = "peer3"; 103 104 private static final String PEER_ID1 = "1"; 105 private static final String PEER_ID3 = "3"; 106 107 private static AtomicInteger BULK_LOADS_COUNT; 108 private static CountDownLatch BULK_LOAD_LATCH; 109 110 protected static final HBaseTestingUtil UTIL3 = new HBaseTestingUtil(); 111 protected static final Configuration CONF3 = UTIL3.getConfiguration(); 112 113 private static final Path BULK_LOAD_BASE_DIR = new Path("/bulk_dir"); 114 115 private static Table htable3; 116 117 @Rule 118 public TestName name = new TestName(); 119 120 @ClassRule 121 public static TemporaryFolder testFolder = new TemporaryFolder(); 122 123 @BeforeClass 124 public static void setUpBeforeClass() throws Exception { 125 setupBulkLoadConfigsForCluster(CONF1, PEER1_CLUSTER_ID); 126 setupBulkLoadConfigsForCluster(CONF2, PEER2_CLUSTER_ID); 127 setupBulkLoadConfigsForCluster(CONF3, PEER3_CLUSTER_ID); 128 setupConfig(UTIL3, "/3"); 129 TestReplicationBase.setUpBeforeClass(); 130 startThirdCluster(); 131 } 132 133 private static void startThirdCluster() throws Exception { 134 LOG.info("Setup Zk to same one from UTIL1 and UTIL2"); 135 UTIL3.setZkCluster(UTIL1.getZkCluster()); 136 UTIL3.startMiniCluster(NUM_SLAVES1); 137 138 TableDescriptor table = TableDescriptorBuilder.newBuilder(tableName) 139 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName).setMobEnabled(true) 140 .setMobThreshold(4000).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()) 141 .setColumnFamily(ColumnFamilyDescriptorBuilder.of(noRepfamName)).build(); 142 143 Connection connection3 = ConnectionFactory.createConnection(CONF3); 144 try (Admin admin3 = connection3.getAdmin()) { 145 admin3.createTable(table, HBaseTestingUtil.KEYS_FOR_HBA_CREATE_TABLE); 146 } 147 UTIL3.waitUntilAllRegionsAssigned(tableName); 148 htable3 = connection3.getTable(tableName); 149 } 150 151 @Before 152 @Override 153 public void setUpBase() throws Exception { 154 // "super.setUpBase()" already sets replication from 1->2, 155 // then on the subsequent lines, sets 2->1, 2->3 and 3->2. 156 // So we have following topology: "1 <-> 2 <->3" 157 super.setUpBase(); 158 ReplicationPeerConfig peer1Config = getPeerConfigForCluster(UTIL1); 159 ReplicationPeerConfig peer2Config = getPeerConfigForCluster(UTIL2); 160 ReplicationPeerConfig peer3Config = getPeerConfigForCluster(UTIL3); 161 // adds cluster1 as a remote peer on cluster2 162 UTIL2.getAdmin().addReplicationPeer(PEER_ID1, peer1Config); 163 // adds cluster3 as a remote peer on cluster2 164 UTIL2.getAdmin().addReplicationPeer(PEER_ID3, peer3Config); 165 // adds cluster2 as a remote peer on cluster3 166 UTIL3.getAdmin().addReplicationPeer(PEER_ID2, peer2Config); 167 setupCoprocessor(UTIL1); 168 setupCoprocessor(UTIL2); 169 setupCoprocessor(UTIL3); 170 BULK_LOADS_COUNT = new AtomicInteger(0); 171 } 172 173 private ReplicationPeerConfig getPeerConfigForCluster(HBaseTestingUtil util) { 174 return ReplicationPeerConfig.newBuilder().setClusterKey(util.getClusterKey()) 175 .setSerial(isSerialPeer()).build(); 176 } 177 178 private void setupCoprocessor(HBaseTestingUtil cluster) { 179 cluster.getHBaseCluster().getRegions(tableName).forEach(r -> { 180 try { 181 TestBulkLoadReplication.BulkReplicationTestObserver cp = r.getCoprocessorHost() 182 .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class); 183 if (cp == null) { 184 r.getCoprocessorHost().load(TestBulkLoadReplication.BulkReplicationTestObserver.class, 0, 185 cluster.getConfiguration()); 186 cp = r.getCoprocessorHost() 187 .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class); 188 cp.clusterName = cluster.getClusterKey(); 189 } 190 } catch (Exception e) { 191 LOG.error(e.getMessage(), e); 192 } 193 }); 194 } 195 196 @After 197 @Override 198 public void tearDownBase() throws Exception { 199 super.tearDownBase(); 200 UTIL2.getAdmin().removeReplicationPeer(PEER_ID1); 201 UTIL2.getAdmin().removeReplicationPeer(PEER_ID3); 202 UTIL3.getAdmin().removeReplicationPeer(PEER_ID2); 203 } 204 205 protected static void setupBulkLoadConfigsForCluster(Configuration config, 206 String clusterReplicationId) throws Exception { 207 config.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); 208 config.set(REPLICATION_CLUSTER_ID, clusterReplicationId); 209 File sourceConfigFolder = testFolder.newFolder(clusterReplicationId); 210 File sourceConfigFile = new File(sourceConfigFolder.getAbsolutePath() + "/hbase-site.xml"); 211 config.writeXml(new FileOutputStream(sourceConfigFile)); 212 config.set(REPLICATION_CONF_DIR, testFolder.getRoot().getAbsolutePath()); 213 } 214 215 @Test 216 public void testBulkLoadReplicationActiveActive() throws Exception { 217 Table peer1TestTable = UTIL1.getConnection().getTable(TestReplicationBase.tableName); 218 Table peer2TestTable = UTIL2.getConnection().getTable(TestReplicationBase.tableName); 219 Table peer3TestTable = UTIL3.getConnection().getTable(TestReplicationBase.tableName); 220 byte[] row = Bytes.toBytes("001"); 221 byte[] value = Bytes.toBytes("v1"); 222 assertBulkLoadConditions(tableName, row, value, UTIL1, peer1TestTable, peer2TestTable, 223 peer3TestTable); 224 row = Bytes.toBytes("002"); 225 value = Bytes.toBytes("v2"); 226 assertBulkLoadConditions(tableName, row, value, UTIL2, peer1TestTable, peer2TestTable, 227 peer3TestTable); 228 row = Bytes.toBytes("003"); 229 value = Bytes.toBytes("v3"); 230 assertBulkLoadConditions(tableName, row, value, UTIL3, peer1TestTable, peer2TestTable, 231 peer3TestTable); 232 // Additional wait to make sure no extra bulk load happens 233 Thread.sleep(400); 234 // We have 3 bulk load events (1 initiated on each cluster). 235 // Each event gets 3 counts (the originator cluster, plus the two peers), 236 // so BULK_LOADS_COUNT expected value is 3 * 3 = 9. 237 assertEquals(9, BULK_LOADS_COUNT.get()); 238 } 239 240 protected void assertBulkLoadConditions(TableName tableName, byte[] row, byte[] value, 241 HBaseTestingUtil utility, Table... tables) throws Exception { 242 BULK_LOAD_LATCH = new CountDownLatch(3); 243 bulkLoadOnCluster(tableName, row, value, utility); 244 assertTrue(BULK_LOAD_LATCH.await(1, TimeUnit.MINUTES)); 245 assertTableHasValue(tables[0], row, value); 246 assertTableHasValue(tables[1], row, value); 247 assertTableHasValue(tables[2], row, value); 248 } 249 250 protected void bulkLoadOnCluster(TableName tableName, byte[] row, byte[] value, 251 HBaseTestingUtil cluster) throws Exception { 252 String bulkLoadFilePath = createHFileForFamilies(row, value, cluster.getConfiguration()); 253 copyToHdfs(bulkLoadFilePath, cluster.getDFSCluster()); 254 BulkLoadHFilesTool bulkLoadHFilesTool = new BulkLoadHFilesTool(cluster.getConfiguration()); 255 bulkLoadHFilesTool.bulkLoad(tableName, BULK_LOAD_BASE_DIR); 256 } 257 258 private void copyToHdfs(String bulkLoadFilePath, MiniDFSCluster cluster) throws Exception { 259 Path bulkLoadDir = new Path(BULK_LOAD_BASE_DIR, "f"); 260 cluster.getFileSystem().mkdirs(bulkLoadDir); 261 cluster.getFileSystem().copyFromLocalFile(new Path(bulkLoadFilePath), bulkLoadDir); 262 } 263 264 protected void assertTableHasValue(Table table, byte[] row, byte[] value) throws Exception { 265 Get get = new Get(row); 266 Result result = table.get(get); 267 assertTrue(result.advance()); 268 assertEquals(Bytes.toString(value), Bytes.toString(result.value())); 269 } 270 271 protected void assertTableNoValue(Table table, byte[] row, byte[] value) throws Exception { 272 Get get = new Get(row); 273 Result result = table.get(get); 274 assertTrue(result.isEmpty()); 275 } 276 277 private String createHFileForFamilies(byte[] row, byte[] value, Configuration clusterConfig) 278 throws IOException { 279 CellBuilder cellBuilder = CellBuilderFactory.create(CellBuilderType.DEEP_COPY); 280 cellBuilder.setRow(row).setFamily(TestReplicationBase.famName).setQualifier(Bytes.toBytes("1")) 281 .setValue(value).setType(Cell.Type.Put); 282 283 HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(clusterConfig); 284 // TODO We need a way to do this without creating files 285 File hFileLocation = testFolder.newFile(); 286 FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null); 287 try { 288 hFileFactory.withOutputStream(out); 289 hFileFactory.withFileContext(new HFileContextBuilder().build()); 290 HFile.Writer writer = hFileFactory.create(); 291 try { 292 writer.append(new KeyValue(cellBuilder.build())); 293 } finally { 294 writer.close(); 295 } 296 } finally { 297 out.close(); 298 } 299 return hFileLocation.getAbsoluteFile().getAbsolutePath(); 300 } 301 302 public static class BulkReplicationTestObserver implements RegionCoprocessor { 303 304 String clusterName; 305 AtomicInteger bulkLoadCounts = new AtomicInteger(); 306 307 @Override 308 public Optional<RegionObserver> getRegionObserver() { 309 return Optional.of(new RegionObserver() { 310 311 @Override 312 public void postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx, 313 List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths) 314 throws IOException { 315 BULK_LOAD_LATCH.countDown(); 316 BULK_LOADS_COUNT.incrementAndGet(); 317 LOG.debug("Another file bulk loaded. Total for {}: {}", clusterName, 318 bulkLoadCounts.addAndGet(1)); 319 } 320 }); 321 } 322 } 323}