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.replication.regionserver;
019
020import static org.junit.jupiter.api.Assertions.assertTrue;
021
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.hbase.HBaseConfiguration;
024import org.apache.hadoop.hbase.HBaseTestingUtil;
025import org.apache.hadoop.hbase.HConstants;
026import org.apache.hadoop.hbase.HTestConst;
027import org.apache.hadoop.hbase.TableName;
028import org.apache.hadoop.hbase.client.Admin;
029import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
030import org.apache.hadoop.hbase.client.Connection;
031import org.apache.hadoop.hbase.client.ConnectionFactory;
032import org.apache.hadoop.hbase.client.Put;
033import org.apache.hadoop.hbase.client.ResultScanner;
034import org.apache.hadoop.hbase.client.Scan;
035import org.apache.hadoop.hbase.client.Table;
036import org.apache.hadoop.hbase.client.TableDescriptor;
037import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
038import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
039import org.apache.hadoop.hbase.testclassification.LargeTests;
040import org.apache.hadoop.hbase.testclassification.ReplicationTests;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.apache.hadoop.hbase.util.Threads;
043import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
044import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
045import org.junit.jupiter.api.AfterAll;
046import org.junit.jupiter.api.BeforeAll;
047import org.junit.jupiter.api.Tag;
048import org.junit.jupiter.api.Test;
049import org.junit.jupiter.api.TestInfo;
050
051import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
052
053@Tag(ReplicationTests.TAG)
054@Tag(LargeTests.TAG)
055public class TestGlobalReplicationThrottler {
056
057  private static final int REPLICATION_SOURCE_QUOTA = 200;
058  private static int numOfPeer = 0;
059  private static Configuration conf1;
060  private static Configuration conf2;
061
062  private static HBaseTestingUtil utility1;
063  private static HBaseTestingUtil utility2;
064
065  private static final byte[] famName = Bytes.toBytes("f");
066  private static final byte[] VALUE = Bytes.toBytes("v");
067  private static final byte[] ROW = Bytes.toBytes("r");
068  private static final byte[][] ROWS = HTestConst.makeNAscii(ROW, 100);
069
070  private String testName;
071
072  @BeforeAll
073  public static void setUpBeforeClass() throws Exception {
074    conf1 = HBaseConfiguration.create();
075    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
076    conf1.setLong("replication.source.sleepforretries", 100);
077    // Each WAL is about 120 bytes
078    conf1.setInt(HConstants.REPLICATION_SOURCE_TOTAL_BUFFER_KEY, REPLICATION_SOURCE_QUOTA);
079    conf1.setLong("replication.source.per.peer.node.bandwidth", 100L);
080
081    utility1 = new HBaseTestingUtil(conf1);
082    utility1.startMiniZKCluster();
083    MiniZooKeeperCluster miniZK = utility1.getZkCluster();
084    new ZKWatcher(conf1, "cluster1", null, true).close();
085
086    conf2 = new Configuration(conf1);
087    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
088
089    utility2 = new HBaseTestingUtil(conf2);
090    utility2.setZkCluster(miniZK);
091    new ZKWatcher(conf2, "cluster2", null, true).close();
092
093    utility1.startMiniCluster();
094    utility2.startMiniCluster();
095
096    ReplicationPeerConfig rpc =
097      ReplicationPeerConfig.newBuilder().setClusterKey(utility2.getRpcConnnectionURI()).build();
098
099    try (Connection connection = ConnectionFactory.createConnection(utility1.getConfiguration());
100      Admin admin1 = connection.getAdmin()) {
101      admin1.addReplicationPeer("peer1", rpc);
102      admin1.addReplicationPeer("peer2", rpc);
103      admin1.addReplicationPeer("peer3", rpc);
104      numOfPeer = admin1.listReplicationPeers().size();
105    }
106  }
107
108  @AfterAll
109  public static void tearDownAfterClass() throws Exception {
110    utility2.shutdownMiniCluster();
111    utility1.shutdownMiniCluster();
112  }
113
114  private volatile boolean testQuotaPass = false;
115  private volatile boolean testQuotaNonZero = false;
116
117  @Test
118  public void testQuota(TestInfo testInfo) throws Exception {
119    testName = testInfo.getTestMethod().get().getName();
120    final TableName tableName = TableName.valueOf(testName);
121    TableDescriptor tableDescriptor =
122      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
123        .newBuilder(famName).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build();
124    utility1.getAdmin().createTable(tableDescriptor);
125    utility2.getAdmin().createTable(tableDescriptor);
126
127    Thread watcher = new Thread(() -> {
128      Replication replication = (Replication) utility1.getMiniHBaseCluster().getRegionServer(0)
129        .getReplicationSourceService();
130      testQuotaPass = true;
131      while (!Thread.interrupted()) {
132        long size = replication.getReplicationManager().getTotalBufferUsed();
133        if (size > 0) {
134          testQuotaNonZero = true;
135        }
136        // the reason here doing "numOfPeer + 1" is because by using method addEntryToBatch(), even
137        // the batch size (after added last entry) exceeds quota, it still keeps the last one in the
138        // batch so total used buffer size can be one "replication.total.buffer.quota" larger than
139        // expected
140        if (size > REPLICATION_SOURCE_QUOTA * (numOfPeer + 1)) {
141          // We read logs first then check throttler, so if the buffer quota limiter doesn't
142          // take effect, it will push many logs and exceed the quota.
143          testQuotaPass = false;
144        }
145        Threads.sleep(50);
146      }
147    });
148    watcher.start();
149
150    try (Table t1 = utility1.getConnection().getTable(tableName)) {
151      for (int i = 0; i < 50; i++) {
152        Put put = new Put(ROWS[i]);
153        put.addColumn(famName, VALUE, VALUE);
154        t1.put(put);
155      }
156    }
157    utility2.waitFor(180000, () -> {
158      try (Table t2 = utility2.getConnection().getTable(tableName);
159        ResultScanner results = t2.getScanner(new Scan().setCaching(50))) {
160        int count = Iterables.size(results);
161        return count >= 50;
162      }
163    });
164
165    watcher.interrupt();
166    assertTrue(testQuotaPass);
167    assertTrue(testQuotaNonZero);
168  }
169
170}