HBASE-22002 Remove the deprecated methods in Admin interface
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / client / TestTableOperationException.java
blob2bb948e528dc0c124087f64547bed66367e1df2f
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.client;
20 import java.io.IOException;
21 import java.util.List;
22 import java.util.Optional;
24 import org.apache.hadoop.hbase.Cell;
25 import org.apache.hadoop.hbase.DoNotRetryIOException;
26 import org.apache.hadoop.hbase.HBaseClassTestRule;
27 import org.apache.hadoop.hbase.HBaseTestingUtility;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
31 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
32 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
33 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
34 import org.apache.hadoop.hbase.testclassification.ClientTests;
35 import org.apache.hadoop.hbase.testclassification.MediumTests;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.wal.WALEdit;
38 import org.junit.AfterClass;
39 import org.junit.BeforeClass;
40 import org.junit.ClassRule;
41 import org.junit.Test;
42 import org.junit.experimental.categories.Category;
44 @Category({MediumTests.class, ClientTests.class})
45 public class TestTableOperationException {
47 @ClassRule
48 public static final HBaseClassTestRule CLASS_RULE =
49 HBaseClassTestRule.forClass(TestTableOperationException.class);
51 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
53 private static TableName TABLE_DONOT_RETRY = TableName.valueOf("TableDoNotRetry");
55 private static TableName TABLE_RETRY = TableName.valueOf("TableRetry");
57 private static Table tableDoNotRetry;
59 private static Table tableRetry;
61 private static byte[] CF = Bytes.toBytes("cf");
63 private static byte[] CQ = Bytes.toBytes("cq");
65 @BeforeClass
66 public static void setUp() throws Exception {
67 UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
68 UTIL.startMiniCluster();
69 UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TABLE_DONOT_RETRY)
70 .setCoprocessor(ThrowDoNotRetryIOExceptionCoprocessor.class.getName())
71 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF).build()).build());
72 UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TABLE_RETRY)
73 .setCoprocessor(ThrowIOExceptionCoprocessor.class.getName())
74 .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF).build()).build());
75 tableDoNotRetry = UTIL.getConnection().getTable(TABLE_DONOT_RETRY);
76 tableRetry = UTIL.getConnection().getTable(TABLE_RETRY);
79 @AfterClass
80 public static void tearDown() throws Exception {
81 UTIL.getAdmin().disableTable(TABLE_DONOT_RETRY);
82 UTIL.getAdmin().disableTable(TABLE_RETRY);
83 UTIL.getAdmin().deleteTable(TABLE_DONOT_RETRY);
84 UTIL.getAdmin().deleteTable(TABLE_RETRY);
85 UTIL.shutdownMiniCluster();
88 @Test(expected = DoNotRetryIOException.class)
89 public void testGetWithDoNotRetryIOException() throws Exception {
90 tableDoNotRetry.get(new Get(Bytes.toBytes("row")).addColumn(CF, CQ));
93 @Test(expected = DoNotRetryIOException.class)
94 public void testPutWithDoNotRetryIOException() throws Exception {
95 tableDoNotRetry.put(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("value")));
98 @Test(expected = DoNotRetryIOException.class)
99 public void testDeleteWithDoNotRetryIOException() throws Exception {
100 tableDoNotRetry.delete(new Delete(Bytes.toBytes("row")).addColumn(CF, CQ));
103 @Test(expected = DoNotRetryIOException.class)
104 public void testAppendWithDoNotRetryIOException() throws Exception {
105 tableDoNotRetry
106 .append(new Append(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("value")));
109 @Test(expected = DoNotRetryIOException.class)
110 public void testIncrementWithDoNotRetryIOException() throws Exception {
111 tableDoNotRetry.increment(new Increment(Bytes.toBytes("row")).addColumn(CF, CQ, 1));
114 @Test(expected = RetriesExhaustedException.class)
115 public void testGetWithIOException() throws Exception {
116 tableRetry.get(new Get(Bytes.toBytes("row")).addColumn(CF, CQ));
119 @Test(expected = RetriesExhaustedException.class)
120 public void testPutWithIOException() throws Exception {
121 tableRetry.put(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("value")));
124 @Test(expected = RetriesExhaustedException.class)
125 public void testDeleteWithIOException() throws Exception {
126 tableRetry.delete(new Delete(Bytes.toBytes("row")).addColumn(CF, CQ));
129 @Test(expected = RetriesExhaustedException.class)
130 public void testAppendWithIOException() throws Exception {
131 tableRetry.append(new Append(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("value")));
134 @Test(expected = RetriesExhaustedException.class)
135 public void testIncrementWithIOException() throws Exception {
136 tableRetry.increment(new Increment(Bytes.toBytes("row")).addColumn(CF, CQ, 1));
139 public static class ThrowDoNotRetryIOExceptionCoprocessor
140 implements RegionCoprocessor, RegionObserver {
142 public ThrowDoNotRetryIOExceptionCoprocessor() {
145 @Override
146 public Optional<RegionObserver> getRegionObserver() {
147 return Optional.of(this);
150 @Override
151 public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e, final Get get,
152 final List<Cell> results) throws IOException {
153 throw new DoNotRetryIOException("Call failed and don't retry");
156 @Override
157 public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put,
158 final WALEdit edit, final Durability durability) throws IOException {
159 throw new DoNotRetryIOException("Call failed and don't retry");
162 @Override
163 public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
164 final Delete delete, final WALEdit edit, final Durability durability) throws IOException {
165 throw new DoNotRetryIOException("Call failed and don't retry");
168 @Override
169 public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
170 final Increment increment) throws IOException {
171 throw new DoNotRetryIOException("Call failed and don't retry");
174 @Override
175 public Result preAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
176 final Append append) throws IOException {
177 throw new DoNotRetryIOException("Call failed and don't retry");
181 public static class ThrowIOExceptionCoprocessor
182 implements RegionCoprocessor, RegionObserver {
184 public ThrowIOExceptionCoprocessor() {
187 @Override
188 public Optional<RegionObserver> getRegionObserver() {
189 return Optional.of(this);
192 @Override
193 public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e, final Get get,
194 final List<Cell> results) throws IOException {
195 throw new IOException("Call failed and retry");
198 @Override
199 public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e, final Put put,
200 final WALEdit edit, final Durability durability) throws IOException {
201 throw new IOException("Call failed and retry");
204 @Override
205 public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e,
206 final Delete delete, final WALEdit edit, final Durability durability) throws IOException {
207 throw new IOException("Call failed and retry");
210 @Override
211 public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
212 final Increment increment) throws IOException {
213 throw new IOException("Call failed and retry");
216 @Override
217 public Result preAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
218 final Append append) throws IOException {
219 throw new IOException("Call failed and retry");