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
.replication
;
20 import static org
.junit
.Assert
.fail
;
22 import java
.io
.IOException
;
24 import org
.apache
.hadoop
.hbase
.HBaseClassTestRule
;
25 import org
.apache
.hadoop
.hbase
.HConstants
;
26 import org
.apache
.hadoop
.hbase
.NamespaceDescriptor
;
27 import org
.apache
.hadoop
.hbase
.TableName
;
28 import org
.apache
.hadoop
.hbase
.client
.Admin
;
29 import org
.apache
.hadoop
.hbase
.client
.ColumnFamilyDescriptorBuilder
;
30 import org
.apache
.hadoop
.hbase
.client
.Connection
;
31 import org
.apache
.hadoop
.hbase
.client
.ConnectionFactory
;
32 import org
.apache
.hadoop
.hbase
.client
.Put
;
33 import org
.apache
.hadoop
.hbase
.client
.Result
;
34 import org
.apache
.hadoop
.hbase
.client
.ResultScanner
;
35 import org
.apache
.hadoop
.hbase
.client
.Scan
;
36 import org
.apache
.hadoop
.hbase
.client
.Table
;
37 import org
.apache
.hadoop
.hbase
.client
.TableDescriptor
;
38 import org
.apache
.hadoop
.hbase
.client
.TableDescriptorBuilder
;
39 import org
.apache
.hadoop
.hbase
.ipc
.RpcServer
;
40 import org
.apache
.hadoop
.hbase
.testclassification
.LargeTests
;
41 import org
.apache
.hadoop
.hbase
.testclassification
.ReplicationTests
;
42 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
43 import org
.apache
.hadoop
.hbase
.util
.JVMClusterUtil
;
44 import org
.junit
.Assert
;
45 import org
.junit
.Before
;
46 import org
.junit
.ClassRule
;
47 import org
.junit
.Test
;
48 import org
.junit
.experimental
.categories
.Category
;
49 import org
.slf4j
.Logger
;
50 import org
.slf4j
.LoggerFactory
;
52 @Category({ ReplicationTests
.class, LargeTests
.class })
53 public class TestReplicationDroppedTables
extends TestReplicationBase
{
56 public static final HBaseClassTestRule CLASS_RULE
=
57 HBaseClassTestRule
.forClass(TestReplicationDroppedTables
.class);
59 private static final Logger LOG
= LoggerFactory
.getLogger(TestReplicationDroppedTables
.class);
60 private static final int ROWS_COUNT
= 1000;
63 public void setUpBase() throws Exception
{
64 // Starting and stopping replication can make us miss new logs,
65 // rolling like this makes sure the most recent one gets added to the queue
66 for (JVMClusterUtil
.RegionServerThread r
: UTIL1
.getHBaseCluster()
67 .getRegionServerThreads()) {
68 UTIL1
.getAdmin().rollWALWriter(r
.getRegionServer().getServerName());
70 // Initialize the peer after wal rolling, so that we will abandon the stuck WALs.
72 int rowCount
= UTIL1
.countRows(tableName
);
73 UTIL1
.deleteTableData(tableName
);
74 // truncating the table will send one Delete per row to the slave cluster
75 // in an async fashion, which is why we cannot just call deleteTableData on
76 // utility2 since late writes could make it to the slave in some way.
77 // Instead, we truncate the first table and wait for all the Deletes to
78 // make it to the slave.
79 Scan scan
= new Scan();
81 for (int i
= 0; i
< NB_RETRIES
; i
++) {
82 if (i
== NB_RETRIES
- 1) {
83 fail("Waited too much time for truncate");
85 ResultScanner scanner
= htable2
.getScanner(scan
);
86 Result
[] res
= scanner
.next(rowCount
);
88 if (res
.length
!= 0) {
89 if (res
.length
< lastCount
) {
90 i
--; // Don't increment timeout if we make progress
92 lastCount
= res
.length
;
93 LOG
.info("Still got " + res
.length
+ " rows");
94 Thread
.sleep(SLEEP_TIME
);
99 // Set the max request size to a tiny 10K for dividing the replication WAL entries into multiple
100 // batches. the default max request size is 256M, so all replication entries are in a batch, but
101 // when replicate at sink side, it'll apply to rs group by table name, so the WAL of test table
102 // may apply first, and then test_dropped table, and we will believe that the replication is not
103 // got stuck (HBASE-20475).
104 CONF1
.setInt(RpcServer
.MAX_REQUEST_SIZE
, 10 * 1024);
108 public void testEditsStuckBehindDroppedTable() throws Exception
{
109 // Sanity check Make sure by default edits for dropped tables stall the replication queue, even
110 // when the table(s) in question have been deleted on both ends.
111 testEditsBehindDroppedTable(false, "test_dropped");
115 public void testEditsDroppedWithDroppedTable() throws Exception
{
116 // Make sure by default edits for dropped tables are themselves dropped when the
117 // table(s) in question have been deleted on both ends.
118 testEditsBehindDroppedTable(true, "test_dropped");
122 public void testEditsDroppedWithDroppedTableNS() throws Exception
{
123 // also try with a namespace
124 UTIL1
.getAdmin().createNamespace(NamespaceDescriptor
.create("NS").build());
125 UTIL2
.getAdmin().createNamespace(NamespaceDescriptor
.create("NS").build());
127 testEditsBehindDroppedTable(true, "NS:test_dropped");
129 UTIL1
.getAdmin().deleteNamespace("NS");
130 UTIL2
.getAdmin().deleteNamespace("NS");
134 private byte[] generateRowKey(int id
) {
135 return Bytes
.toBytes(String
.format("NormalPut%03d", id
));
138 private void testEditsBehindDroppedTable(boolean allowProceeding
, String tName
) throws Exception
{
139 CONF1
.setBoolean(HConstants
.REPLICATION_DROP_ON_DELETED_TABLE_KEY
, allowProceeding
);
140 CONF1
.setInt(HConstants
.REPLICATION_SOURCE_MAXTHREADS_KEY
, 1);
142 // make sure we have a single region server only, so that all
143 // edits for all tables go there
144 restartSourceCluster(1);
146 TableName tablename
= TableName
.valueOf(tName
);
147 byte[] familyName
= Bytes
.toBytes("fam");
148 byte[] row
= Bytes
.toBytes("row");
150 TableDescriptor table
=
151 TableDescriptorBuilder
152 .newBuilder(tablename
).setColumnFamily(ColumnFamilyDescriptorBuilder
153 .newBuilder(familyName
).setScope(HConstants
.REPLICATION_SCOPE_GLOBAL
).build())
156 Connection connection1
= ConnectionFactory
.createConnection(UTIL1
.getConfiguration());
157 Connection connection2
= ConnectionFactory
.createConnection(UTIL2
.getConfiguration());
158 try (Admin admin1
= connection1
.getAdmin()) {
159 admin1
.createTable(table
);
161 try (Admin admin2
= connection2
.getAdmin()) {
162 admin2
.createTable(table
);
164 UTIL1
.waitUntilAllRegionsAssigned(tablename
);
165 UTIL2
.waitUntilAllRegionsAssigned(tablename
);
167 // now suspend replication
168 try (Admin admin1
= connection1
.getAdmin()) {
169 admin1
.disableReplicationPeer(PEER_ID2
);
172 // put some data (lead with 0 so the edit gets sorted before the other table's edits
173 // in the replication batch) write a bunch of edits, making sure we fill a batch
174 try (Table droppedTable
= connection1
.getTable(tablename
)) {
175 byte[] rowKey
= Bytes
.toBytes(0 + " put on table to be dropped");
176 Put put
= new Put(rowKey
);
177 put
.addColumn(familyName
, row
, row
);
178 droppedTable
.put(put
);
181 try (Table table1
= connection1
.getTable(tableName
)) {
182 for (int i
= 0; i
< ROWS_COUNT
; i
++) {
183 Put put
= new Put(generateRowKey(i
)).addColumn(famName
, row
, row
);
188 try (Admin admin1
= connection1
.getAdmin()) {
189 admin1
.disableTable(tablename
);
190 admin1
.deleteTable(tablename
);
192 try (Admin admin2
= connection2
.getAdmin()) {
193 admin2
.disableTable(tablename
);
194 admin2
.deleteTable(tablename
);
197 try (Admin admin1
= connection1
.getAdmin()) {
198 admin1
.enableReplicationPeer(PEER_ID2
);
201 if (allowProceeding
) {
202 // in this we'd expect the key to make it over
203 verifyReplicationProceeded();
205 verifyReplicationStuck();
208 CONF1
.setBoolean(HConstants
.REPLICATION_DROP_ON_DELETED_TABLE_KEY
, false);
212 public void testEditsBehindDroppedTableTiming() throws Exception
{
213 CONF1
.setBoolean(HConstants
.REPLICATION_DROP_ON_DELETED_TABLE_KEY
, true);
214 CONF1
.setInt(HConstants
.REPLICATION_SOURCE_MAXTHREADS_KEY
, 1);
216 // make sure we have a single region server only, so that all
217 // edits for all tables go there
218 restartSourceCluster(1);
220 TableName tablename
= TableName
.valueOf("testdroppedtimed");
221 byte[] familyName
= Bytes
.toBytes("fam");
222 byte[] row
= Bytes
.toBytes("row");
224 TableDescriptor table
=
225 TableDescriptorBuilder
226 .newBuilder(tablename
).setColumnFamily(ColumnFamilyDescriptorBuilder
227 .newBuilder(familyName
).setScope(HConstants
.REPLICATION_SCOPE_GLOBAL
).build())
230 Connection connection1
= ConnectionFactory
.createConnection(CONF1
);
231 Connection connection2
= ConnectionFactory
.createConnection(CONF2
);
232 try (Admin admin1
= connection1
.getAdmin()) {
233 admin1
.createTable(table
);
235 try (Admin admin2
= connection2
.getAdmin()) {
236 admin2
.createTable(table
);
238 UTIL1
.waitUntilAllRegionsAssigned(tablename
);
239 UTIL2
.waitUntilAllRegionsAssigned(tablename
);
241 // now suspend replication
242 try (Admin admin1
= connection1
.getAdmin()) {
243 admin1
.disableReplicationPeer(PEER_ID2
);
246 // put some data (lead with 0 so the edit gets sorted before the other table's edits
247 // in the replication batch) write a bunch of edits, making sure we fill a batch
248 try (Table droppedTable
= connection1
.getTable(tablename
)) {
249 byte[] rowKey
= Bytes
.toBytes(0 + " put on table to be dropped");
250 Put put
= new Put(rowKey
);
251 put
.addColumn(familyName
, row
, row
);
252 droppedTable
.put(put
);
255 try (Table table1
= connection1
.getTable(tableName
)) {
256 for (int i
= 0; i
< ROWS_COUNT
; i
++) {
257 Put put
= new Put(generateRowKey(i
)).addColumn(famName
, row
, row
);
262 try (Admin admin2
= connection2
.getAdmin()) {
263 admin2
.disableTable(tablename
);
264 admin2
.deleteTable(tablename
);
267 // edit should still be stuck
268 try (Admin admin1
= connection1
.getAdmin()) {
269 // enable the replication peer.
270 admin1
.enableReplicationPeer(PEER_ID2
);
271 // the source table still exists, replication should be stalled
272 verifyReplicationStuck();
274 admin1
.disableTable(tablename
);
275 // still stuck, source table still exists
276 verifyReplicationStuck();
278 admin1
.deleteTable(tablename
);
279 // now the source table is gone, replication should proceed, the
280 // offending edits be dropped
281 verifyReplicationProceeded();
284 CONF1
.setBoolean(HConstants
.REPLICATION_DROP_ON_DELETED_TABLE_KEY
, false);
287 private boolean peerHasAllNormalRows() throws IOException
{
288 try (ResultScanner scanner
= htable2
.getScanner(new Scan())) {
289 Result
[] results
= scanner
.next(ROWS_COUNT
);
290 if (results
.length
!= ROWS_COUNT
) {
293 for (int i
= 0; i
< results
.length
; i
++) {
294 Assert
.assertArrayEquals(generateRowKey(i
), results
[i
].getRow());
300 private void verifyReplicationProceeded() throws Exception
{
301 for (int i
= 0; i
< NB_RETRIES
; i
++) {
302 if (i
== NB_RETRIES
- 1) {
303 fail("Waited too much time for put replication");
305 if (!peerHasAllNormalRows()) {
306 LOG
.info("Row not available");
307 Thread
.sleep(SLEEP_TIME
);
314 private void verifyReplicationStuck() throws Exception
{
315 for (int i
= 0; i
< NB_RETRIES
; i
++) {
316 if (peerHasAllNormalRows()) {
317 fail("Edit should have been stuck behind dropped tables");
319 LOG
.info("Row not replicated, let's wait a bit more...");
320 Thread
.sleep(SLEEP_TIME
);