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 java.io.IOException; 021import java.util.concurrent.CompletableFuture; 022import java.util.concurrent.CountDownLatch; 023import java.util.concurrent.atomic.AtomicLong; 024import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; 025import org.apache.hadoop.hbase.testclassification.ClientTests; 026import org.apache.hadoop.hbase.testclassification.LargeTests; 027import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 028import org.apache.hadoop.hbase.util.FutureUtils; 029import org.junit.jupiter.api.AfterEach; 030import org.junit.jupiter.api.BeforeEach; 031import org.junit.jupiter.api.Tag; 032 033import org.apache.hbase.thirdparty.com.google.common.io.Closeables; 034 035@Tag(LargeTests.TAG) 036@Tag(ClientTests.TAG) 037public class TestAsyncClientPushback extends ClientPushbackTestBase { 038 039 private AsyncConnectionImpl conn; 040 041 private AsyncBufferedMutator mutator; 042 043 @BeforeEach 044 public void setUp() throws Exception { 045 conn = 046 (AsyncConnectionImpl) ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get(); 047 mutator = conn.getBufferedMutator(tableName); 048 } 049 050 @AfterEach 051 public void tearDown() throws IOException { 052 Closeables.close(mutator, true); 053 Closeables.close(conn, true); 054 } 055 056 @Override 057 protected ClientBackoffPolicy getBackoffPolicy() throws IOException { 058 return conn.getBackoffPolicy(); 059 } 060 061 @Override 062 protected ServerStatisticTracker getStatisticsTracker() throws IOException { 063 return conn.getStatisticsTracker().get(); 064 } 065 066 @Override 067 protected MetricsConnection getConnectionMetrics() throws IOException { 068 return conn.getConnectionMetrics().get(); 069 } 070 071 @Override 072 protected void mutate(Put put) throws IOException { 073 CompletableFuture<?> future = mutator.mutate(put); 074 mutator.flush(); 075 future.join(); 076 } 077 078 @Override 079 protected void mutate(Put put, AtomicLong endTime, CountDownLatch latch) throws IOException { 080 FutureUtils.addListener(mutator.mutate(put), (r, e) -> { 081 endTime.set(EnvironmentEdgeManager.currentTime()); 082 latch.countDown(); 083 }); 084 mutator.flush(); 085 } 086 087 @Override 088 protected void mutateRow(RowMutations mutations) throws IOException { 089 conn.getTable(tableName).mutateRow(mutations).join(); 090 } 091}