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.HBaseTestingUtility;
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 HBaseTestingUtility UTIL3 = new HBaseTestingUtility();
111  protected static final Configuration CONF3 = UTIL3.getConfiguration();
112
113  private static final Path BULK_LOAD_BASE_DIR = new Path("/bulk_dir");
114
115  @Rule
116  public TestName name = new TestName();
117
118  @ClassRule
119  public static TemporaryFolder testFolder = new TemporaryFolder();
120
121  @BeforeClass
122  public static void setUpBeforeClass() throws Exception {
123    setupBulkLoadConfigsForCluster(CONF1, PEER1_CLUSTER_ID);
124    setupBulkLoadConfigsForCluster(CONF2, PEER2_CLUSTER_ID);
125    setupBulkLoadConfigsForCluster(CONF3, PEER3_CLUSTER_ID);
126    setupConfig(UTIL3, "/3");
127    TestReplicationBase.setUpBeforeClass();
128    startThirdCluster();
129  }
130
131  private static void startThirdCluster() throws Exception {
132    LOG.info("Setup Zk to same one from UTIL1 and UTIL2");
133    UTIL3.setZkCluster(UTIL1.getZkCluster());
134    UTIL3.startMiniCluster(NUM_SLAVES1);
135
136    TableDescriptor table = TableDescriptorBuilder.newBuilder(tableName)
137      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(famName).setMobEnabled(true)
138        .setMobThreshold(4000).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
139      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(noRepfamName)).build();
140
141    Connection connection3 = ConnectionFactory.createConnection(CONF3);
142    try (Admin admin3 = connection3.getAdmin()) {
143      admin3.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
144    }
145    UTIL3.waitUntilAllRegionsAssigned(tableName);
146  }
147
148  @Before
149  @Override
150  public void setUpBase() throws Exception {
151    // "super.setUpBase()" already sets replication from 1->2,
152    // then on the subsequent lines, sets 2->1, 2->3 and 3->2.
153    // So we have following topology: "1 <-> 2 <->3"
154    super.setUpBase();
155    ReplicationPeerConfig peer1Config = getPeerConfigForCluster(UTIL1);
156    ReplicationPeerConfig peer2Config = getPeerConfigForCluster(UTIL2);
157    ReplicationPeerConfig peer3Config = getPeerConfigForCluster(UTIL3);
158    // adds cluster1 as a remote peer on cluster2
159    UTIL2.getAdmin().addReplicationPeer(PEER_ID1, peer1Config);
160    // adds cluster3 as a remote peer on cluster2
161    UTIL2.getAdmin().addReplicationPeer(PEER_ID3, peer3Config);
162    // adds cluster2 as a remote peer on cluster3
163    UTIL3.getAdmin().addReplicationPeer(PEER_ID2, peer2Config);
164    setupCoprocessor(UTIL1);
165    setupCoprocessor(UTIL2);
166    setupCoprocessor(UTIL3);
167    BULK_LOADS_COUNT = new AtomicInteger(0);
168  }
169
170  private ReplicationPeerConfig getPeerConfigForCluster(HBaseTestingUtility util) {
171    return ReplicationPeerConfig.newBuilder().setClusterKey(util.getClusterKey())
172      .setSerial(isSerialPeer()).build();
173  }
174
175  private void setupCoprocessor(HBaseTestingUtility cluster) {
176    cluster.getHBaseCluster().getRegions(tableName).forEach(r -> {
177      try {
178        TestBulkLoadReplication.BulkReplicationTestObserver cp = r.getCoprocessorHost()
179          .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class);
180        if (cp == null) {
181          r.getCoprocessorHost().load(TestBulkLoadReplication.BulkReplicationTestObserver.class, 0,
182            cluster.getConfiguration());
183          cp = r.getCoprocessorHost()
184            .findCoprocessor(TestBulkLoadReplication.BulkReplicationTestObserver.class);
185          cp.clusterName = cluster.getClusterKey();
186        }
187      } catch (Exception e) {
188        LOG.error(e.getMessage(), e);
189      }
190    });
191  }
192
193  @After
194  @Override
195  public void tearDownBase() throws Exception {
196    super.tearDownBase();
197    UTIL2.getAdmin().removeReplicationPeer(PEER_ID1);
198    UTIL2.getAdmin().removeReplicationPeer(PEER_ID3);
199    UTIL3.getAdmin().removeReplicationPeer(PEER_ID2);
200  }
201
202  protected static void setupBulkLoadConfigsForCluster(Configuration config,
203    String clusterReplicationId) throws Exception {
204    config.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
205    config.set(REPLICATION_CLUSTER_ID, clusterReplicationId);
206    File sourceConfigFolder = testFolder.newFolder(clusterReplicationId);
207    File sourceConfigFile = new File(sourceConfigFolder.getAbsolutePath() + "/hbase-site.xml");
208    config.writeXml(new FileOutputStream(sourceConfigFile));
209    config.set(REPLICATION_CONF_DIR, testFolder.getRoot().getAbsolutePath());
210  }
211
212  @Test
213  public void testBulkLoadReplicationActiveActive() throws Exception {
214    Table peer1TestTable = UTIL1.getConnection().getTable(TestReplicationBase.tableName);
215    Table peer2TestTable = UTIL2.getConnection().getTable(TestReplicationBase.tableName);
216    Table peer3TestTable = UTIL3.getConnection().getTable(TestReplicationBase.tableName);
217    byte[] row = Bytes.toBytes("001");
218    byte[] value = Bytes.toBytes("v1");
219    assertBulkLoadConditions(tableName, row, value, UTIL1, peer1TestTable, peer2TestTable,
220      peer3TestTable);
221    row = Bytes.toBytes("002");
222    value = Bytes.toBytes("v2");
223    assertBulkLoadConditions(tableName, row, value, UTIL2, peer1TestTable, peer2TestTable,
224      peer3TestTable);
225    row = Bytes.toBytes("003");
226    value = Bytes.toBytes("v3");
227    assertBulkLoadConditions(tableName, row, value, UTIL3, peer1TestTable, peer2TestTable,
228      peer3TestTable);
229    // Additional wait to make sure no extra bulk load happens
230    Thread.sleep(400);
231    // We have 3 bulk load events (1 initiated on each cluster).
232    // Each event gets 3 counts (the originator cluster, plus the two peers),
233    // so BULK_LOADS_COUNT expected value is 3 * 3 = 9.
234    assertEquals(9, BULK_LOADS_COUNT.get());
235  }
236
237  protected void assertBulkLoadConditions(TableName tableName, byte[] row, byte[] value,
238    HBaseTestingUtility utility, Table... tables) throws Exception {
239    BULK_LOAD_LATCH = new CountDownLatch(3);
240    bulkLoadOnCluster(tableName, row, value, utility);
241    assertTrue(BULK_LOAD_LATCH.await(1, TimeUnit.MINUTES));
242    assertTableHasValue(tables[0], row, value);
243    assertTableHasValue(tables[1], row, value);
244    assertTableHasValue(tables[2], row, value);
245  }
246
247  protected void bulkLoadOnCluster(TableName tableName, byte[] row, byte[] value,
248    HBaseTestingUtility cluster) throws Exception {
249    String bulkLoadFilePath = createHFileForFamilies(row, value, cluster.getConfiguration());
250    copyToHdfs(bulkLoadFilePath, cluster.getDFSCluster());
251    BulkLoadHFilesTool bulkLoadHFilesTool = new BulkLoadHFilesTool(cluster.getConfiguration());
252    bulkLoadHFilesTool.bulkLoad(tableName, BULK_LOAD_BASE_DIR);
253  }
254
255  private void copyToHdfs(String bulkLoadFilePath, MiniDFSCluster cluster) throws Exception {
256    Path bulkLoadDir = new Path(BULK_LOAD_BASE_DIR, "f");
257    cluster.getFileSystem().mkdirs(bulkLoadDir);
258    cluster.getFileSystem().copyFromLocalFile(new Path(bulkLoadFilePath), bulkLoadDir);
259  }
260
261  protected void assertTableHasValue(Table table, byte[] row, byte[] value) throws Exception {
262    Get get = new Get(row);
263    Result result = table.get(get);
264    assertTrue(result.advance());
265    assertEquals(Bytes.toString(value), Bytes.toString(result.value()));
266  }
267
268  protected void assertTableNoValue(Table table, byte[] row, byte[] value) throws Exception {
269    Get get = new Get(row);
270    Result result = table.get(get);
271    assertTrue(result.isEmpty());
272  }
273
274  private String createHFileForFamilies(byte[] row, byte[] value, Configuration clusterConfig)
275    throws IOException {
276    CellBuilder cellBuilder = CellBuilderFactory.create(CellBuilderType.DEEP_COPY);
277    cellBuilder.setRow(row).setFamily(TestReplicationBase.famName).setQualifier(Bytes.toBytes("1"))
278      .setValue(value).setType(Cell.Type.Put);
279
280    HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(clusterConfig);
281    // TODO We need a way to do this without creating files
282    File hFileLocation = testFolder.newFile();
283    FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null);
284    try {
285      hFileFactory.withOutputStream(out);
286      hFileFactory.withFileContext(new HFileContextBuilder().build());
287      HFile.Writer writer = hFileFactory.create();
288      try {
289        writer.append(new KeyValue(cellBuilder.build()));
290      } finally {
291        writer.close();
292      }
293    } finally {
294      out.close();
295    }
296    return hFileLocation.getAbsoluteFile().getAbsolutePath();
297  }
298
299  public static class BulkReplicationTestObserver implements RegionCoprocessor {
300
301    String clusterName;
302    AtomicInteger bulkLoadCounts = new AtomicInteger();
303
304    @Override
305    public Optional<RegionObserver> getRegionObserver() {
306      return Optional.of(new RegionObserver() {
307
308        @Override
309        public void postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
310          List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths)
311          throws IOException {
312          BULK_LOAD_LATCH.countDown();
313          BULK_LOADS_COUNT.incrementAndGet();
314          LOG.debug("Another file bulk loaded. Total for {}: {}", clusterName,
315            bulkLoadCounts.addAndGet(1));
316        }
317      });
318    }
319  }
320}