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.client;
019
020import static org.apache.hadoop.hbase.HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT;
021import static org.apache.hadoop.hbase.io.ByteBuffAllocator.MAX_BUFFER_COUNT_KEY;
022import static org.apache.hadoop.hbase.master.LoadBalancer.TABLES_ON_MASTER;
023import static org.junit.Assert.assertEquals;
024
025import java.io.IOException;
026import java.util.ArrayList;
027import java.util.Arrays;
028import java.util.Collections;
029import java.util.List;
030import java.util.concurrent.ExecutionException;
031import java.util.concurrent.ExecutorService;
032import java.util.concurrent.Executors;
033import java.util.concurrent.Future;
034import java.util.concurrent.ThreadLocalRandom;
035import java.util.concurrent.TimeUnit;
036import java.util.concurrent.atomic.AtomicBoolean;
037import java.util.stream.Collectors;
038import java.util.stream.IntStream;
039import org.apache.hadoop.hbase.HBaseClassTestRule;
040import org.apache.hadoop.hbase.HBaseTestingUtility;
041import org.apache.hadoop.hbase.MemoryCompactionPolicy;
042import org.apache.hadoop.hbase.ServerName;
043import org.apache.hadoop.hbase.TableName;
044import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
045import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
046import org.apache.hadoop.hbase.regionserver.HRegion;
047import org.apache.hadoop.hbase.testclassification.ClientTests;
048import org.apache.hadoop.hbase.testclassification.LargeTests;
049import org.apache.hadoop.hbase.util.Bytes;
050import org.apache.hadoop.hbase.util.RetryCounter;
051import org.apache.hadoop.hbase.util.Threads;
052import org.junit.AfterClass;
053import org.junit.Assert;
054import org.junit.BeforeClass;
055import org.junit.ClassRule;
056import org.junit.Test;
057import org.junit.experimental.categories.Category;
058import org.slf4j.Logger;
059import org.slf4j.LoggerFactory;
060
061import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
062import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
063
064/**
065 * Will split the table, and move region randomly when testing.
066 */
067@Category({ LargeTests.class, ClientTests.class })
068public class TestAsyncTableGetMultiThreaded {
069
070  @ClassRule
071  public static final HBaseClassTestRule CLASS_RULE =
072    HBaseClassTestRule.forClass(TestAsyncTableGetMultiThreaded.class);
073
074  private static final Logger LOG = LoggerFactory.getLogger(TestAsyncTableGetMultiThreaded.class);
075
076  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
077
078  private static final TableName TABLE_NAME = TableName.valueOf("async");
079  private static final byte[] FAMILY = Bytes.toBytes("cf");
080  private static final byte[] QUALIFIER = Bytes.toBytes("cq");
081  private static final int COUNT = 1000;
082
083  private static AsyncConnection CONN;
084
085  private static AsyncTable<?> TABLE;
086
087  private static byte[][] SPLIT_KEYS;
088
089  @BeforeClass
090  public static void setUp() throws Exception {
091    setUp(MemoryCompactionPolicy.NONE);
092  }
093
094  protected static void setUp(MemoryCompactionPolicy memoryCompaction) throws Exception {
095    TEST_UTIL.getConfiguration().set(TABLES_ON_MASTER, "none");
096    TEST_UTIL.getConfiguration().setLong(HBASE_CLIENT_META_OPERATION_TIMEOUT, 60000L);
097    TEST_UTIL.getConfiguration().setInt(MAX_BUFFER_COUNT_KEY, 100);
098    TEST_UTIL.getConfiguration().set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
099      String.valueOf(memoryCompaction));
100    TEST_UTIL.getConfiguration().setBoolean("hbase.master.balancer.decision.buffer.enabled", true);
101
102    TEST_UTIL.startMiniCluster(3);
103    SPLIT_KEYS = new byte[8][];
104    for (int i = 111; i < 999; i += 111) {
105      SPLIT_KEYS[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
106    }
107    TEST_UTIL.createTable(TABLE_NAME, FAMILY);
108    TEST_UTIL.waitTableAvailable(TABLE_NAME);
109    CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
110    TABLE = CONN.getTableBuilder(TABLE_NAME).setReadRpcTimeout(1, TimeUnit.SECONDS)
111      .setMaxRetries(1000).build();
112    TABLE.putAll(
113      IntStream.range(0, COUNT).mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
114        .addColumn(FAMILY, QUALIFIER, Bytes.toBytes(i))).collect(Collectors.toList()))
115      .get();
116  }
117
118  @AfterClass
119  public static void tearDown() throws Exception {
120    Closeables.close(CONN, true);
121    TEST_UTIL.shutdownMiniCluster();
122  }
123
124  private void run(AtomicBoolean stop) throws InterruptedException, ExecutionException {
125    while (!stop.get()) {
126      for (int i = 0; i < COUNT; i++) {
127        assertEquals(i, Bytes.toInt(TABLE.get(new Get(Bytes.toBytes(String.format("%03d", i))))
128          .get().getValue(FAMILY, QUALIFIER)));
129      }
130      // sleep a bit so we do not add to much load to the test machine as we have 20 threads here
131      Thread.sleep(10);
132    }
133  }
134
135  @Test
136  public void test() throws Exception {
137    LOG.info("====== Test started ======");
138    int numThreads = 7;
139    AtomicBoolean stop = new AtomicBoolean(false);
140    ExecutorService executor = Executors.newFixedThreadPool(numThreads,
141      new ThreadFactoryBuilder().setNameFormat("TestAsyncGet-pool-%d").setDaemon(true)
142        .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
143    List<Future<?>> futures = new ArrayList<>();
144    IntStream.range(0, numThreads).forEach(i -> futures.add(executor.submit(() -> {
145      run(stop);
146      return null;
147    })));
148    LOG.info("====== Scheduled {} read threads ======", numThreads);
149    Collections.shuffle(Arrays.asList(SPLIT_KEYS), ThreadLocalRandom.current());
150    Admin admin = TEST_UTIL.getAdmin();
151    for (byte[] splitPoint : SPLIT_KEYS) {
152      int oldRegionCount = admin.getRegions(TABLE_NAME).size();
153      LOG.info("====== Splitting at {} ======, region count before splitting is {}",
154        Bytes.toStringBinary(splitPoint), oldRegionCount);
155      admin.split(TABLE_NAME, splitPoint);
156      TEST_UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
157        @Override
158        public boolean evaluate() throws Exception {
159          return TEST_UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).size() > oldRegionCount;
160        }
161
162        @Override
163        public String explainFailure() throws Exception {
164          return "Split has not finished yet";
165        }
166      });
167      List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME);
168      LOG.info("====== Split at {} ======, region count after splitting is {}",
169        Bytes.toStringBinary(splitPoint), regions.size());
170      for (HRegion region : regions) {
171        LOG.info("====== Compact {} ======", region.getRegionInfo());
172        region.compact(true);
173      }
174      for (HRegion region : regions) {
175        // Waiting for compaction to complete and references are cleaned up
176        LOG.info("====== Waiting for compaction on {} ======", region.getRegionInfo());
177        RetryCounter retrier = new RetryCounter(30, 1, TimeUnit.SECONDS);
178        for (;;) {
179          try {
180            if (
181              admin.getCompactionStateForRegion(region.getRegionInfo().getRegionName())
182                  == CompactionState.NONE
183            ) {
184              break;
185            }
186          } catch (IOException e) {
187            LOG.warn("Failed to query");
188          }
189          if (!retrier.shouldRetry()) {
190            throw new IOException("Can not finish compaction in time after attempt "
191              + retrier.getAttemptTimes() + " times");
192          }
193          retrier.sleepUntilNextRetry();
194        }
195        LOG.info("====== Compaction on {} finished, close and archive compacted files ======",
196          region.getRegionInfo());
197        region.getStores().get(0).closeAndArchiveCompactedFiles();
198        LOG.info("====== Close and archive compacted files on {} done ======",
199          region.getRegionInfo());
200      }
201      Thread.sleep(5000);
202      LOG.info("====== Balancing cluster ======");
203      admin.balance(BalanceRequest.newBuilder().setIgnoreRegionsInTransition(true).build());
204      LOG.info("====== Balance cluster done ======");
205      Thread.sleep(5000);
206      ServerName metaServer = TEST_UTIL.getHBaseCluster().getServerHoldingMeta();
207      ServerName newMetaServer = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
208        .map(t -> t.getRegionServer().getServerName()).filter(s -> !s.equals(metaServer)).findAny()
209        .get();
210      LOG.info("====== Moving meta from {} to {} ======", metaServer, newMetaServer);
211      admin.move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), newMetaServer);
212      LOG.info("====== Move meta done ======");
213      Thread.sleep(5000);
214    }
215    List<LogEntry> balancerDecisionRecords =
216      admin.getLogEntries(null, "BALANCER_DECISION", ServerType.MASTER, 2, null);
217    Assert.assertEquals(balancerDecisionRecords.size(), 2);
218    LOG.info("====== Read test finished, shutdown thread pool ======");
219    stop.set(true);
220    executor.shutdown();
221    for (int i = 0; i < numThreads; i++) {
222      LOG.info("====== Waiting for {} threads to finish, remaining {} ======", numThreads,
223        numThreads - i);
224      futures.get(i).get();
225    }
226    LOG.info("====== Test test finished ======");
227  }
228}