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; 019 020import static org.apache.hadoop.hbase.replication.ReplicationPeerConfigTestUtil.getConfig; 021import static org.junit.jupiter.api.Assertions.assertEquals; 022import static org.junit.jupiter.api.Assertions.assertFalse; 023import static org.junit.jupiter.api.Assertions.assertNull; 024import static org.junit.jupiter.api.Assertions.assertTrue; 025 026import java.util.List; 027import java.util.Map; 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.hbase.HBaseConfiguration; 030import org.apache.hadoop.hbase.TableName; 031import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; 032import org.apache.hadoop.hbase.testclassification.ClientTests; 033import org.apache.hadoop.hbase.testclassification.SmallTests; 034import org.apache.hadoop.hbase.util.BuilderStyleTest; 035import org.apache.hadoop.hbase.util.Bytes; 036import org.junit.jupiter.api.Tag; 037import org.junit.jupiter.api.Test; 038 039import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 040import org.apache.hbase.thirdparty.com.google.common.collect.Maps; 041import org.apache.hbase.thirdparty.com.google.common.collect.Sets; 042 043@Tag(ClientTests.TAG) 044@Tag(SmallTests.TAG) 045public class TestReplicationPeerConfig { 046 047 private static final Configuration CONF = HBaseConfiguration.create(); 048 049 private static final String NAMESPACE_REPLICATE = "replicate"; 050 private static final String NAMESPACE_OTHER = "other"; 051 private static final TableName TABLE_A = TableName.valueOf(NAMESPACE_REPLICATE, "testA"); 052 private static final TableName TABLE_B = TableName.valueOf(NAMESPACE_REPLICATE, "testB"); 053 private static final byte[] FAMILY1 = Bytes.toBytes("cf1"); 054 private static final byte[] FAMILY2 = Bytes.toBytes("cf2"); 055 056 @Test 057 public void testClassMethodsAreBuilderStyle() { 058 /* 059 * ReplicationPeerConfig should have a builder style setup where setXXX/addXXX methods can be 060 * chainable together: . For example: ReplicationPeerConfig htd = new ReplicationPeerConfig() 061 * .setFoo(foo) .setBar(bar) .setBuz(buz) This test ensures that all methods starting with "set" 062 * returns the declaring object 063 */ 064 065 BuilderStyleTest.assertClassesAreBuilderStyle(ReplicationPeerConfig.class); 066 } 067 068 @Test 069 public void testNeedToReplicateWithReplicatingAll() { 070 // 1. replication_all flag is true, no namespaces and table-cfs config 071 ReplicationPeerConfig peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 072 .setReplicateAllUserTables(true).build(); 073 assertTrue(peerConfig.needToReplicate(TABLE_A)); 074 075 // 2. replicate_all flag is true, and config in excludedTableCfs 076 // Exclude empty table-cfs map 077 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 078 .setReplicateAllUserTables(true).setExcludeTableCFsMap(Maps.newHashMap()).build(); 079 assertTrue(peerConfig.needToReplicate(TABLE_A)); 080 081 // Exclude table B 082 Map<TableName, List<String>> tableCfs = Maps.newHashMap(); 083 tableCfs.put(TABLE_B, null); 084 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 085 .setReplicateAllUserTables(true).setExcludeTableCFsMap(tableCfs).build(); 086 assertTrue(peerConfig.needToReplicate(TABLE_A)); 087 assertFalse(peerConfig.needToReplicate(TABLE_B)); 088 089 // 3. replicate_all flag is true, and config in excludeNamespaces 090 // Exclude empty namespace set 091 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 092 .setReplicateAllUserTables(true).setExcludeNamespaces(Sets.newHashSet()).build(); 093 assertTrue(peerConfig.needToReplicate(TABLE_A)); 094 095 // Exclude namespace other 096 peerConfig = 097 new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl().setReplicateAllUserTables(true) 098 .setExcludeNamespaces(Sets.newHashSet(NAMESPACE_OTHER)).build(); 099 assertTrue(peerConfig.needToReplicate(TABLE_A)); 100 101 // Exclude namespace replication 102 peerConfig = 103 new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl().setReplicateAllUserTables(true) 104 .setExcludeNamespaces(Sets.newHashSet(NAMESPACE_REPLICATE)).build(); 105 assertFalse(peerConfig.needToReplicate(TABLE_A)); 106 107 // 4. replicate_all flag is true, and config excludeNamespaces and excludedTableCfs both 108 // Namespaces config doesn't conflict with table-cfs config 109 tableCfs = Maps.newHashMap(); 110 tableCfs.put(TABLE_A, null); 111 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 112 .setReplicateAllUserTables(true).setExcludeNamespaces(Sets.newHashSet(NAMESPACE_REPLICATE)) 113 .setExcludeTableCFsMap(tableCfs).build(); 114 assertFalse(peerConfig.needToReplicate(TABLE_A)); 115 116 // Namespaces config conflicts with table-cfs config 117 tableCfs = Maps.newHashMap(); 118 tableCfs.put(TABLE_A, null); 119 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 120 .setReplicateAllUserTables(true).setExcludeTableCFsMap(tableCfs) 121 .setExcludeNamespaces(Sets.newHashSet(NAMESPACE_OTHER)).build(); 122 assertFalse(peerConfig.needToReplicate(TABLE_A)); 123 assertTrue(peerConfig.needToReplicate(TABLE_B)); 124 125 tableCfs = Maps.newHashMap(); 126 tableCfs.put(TABLE_B, null); 127 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 128 .setReplicateAllUserTables(true).setExcludeTableCFsMap(tableCfs) 129 .setExcludeNamespaces(Sets.newHashSet(NAMESPACE_REPLICATE)).build(); 130 assertFalse(peerConfig.needToReplicate(TABLE_A)); 131 assertFalse(peerConfig.needToReplicate(TABLE_B)); 132 } 133 134 @Test 135 public void testNeedToReplicateWithoutReplicatingAll() { 136 ReplicationPeerConfig peerConfig; 137 Map<TableName, List<String>> tableCfs; 138 139 // 1. replication_all flag is false, no namespaces and table-cfs config 140 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 141 .setReplicateAllUserTables(false).build(); 142 assertFalse(peerConfig.needToReplicate(TABLE_A)); 143 144 // 2. replicate_all flag is false, and only config table-cfs in peer 145 // Set empty table-cfs map 146 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 147 .setReplicateAllUserTables(false).setTableCFsMap(Maps.newHashMap()).build(); 148 assertFalse(peerConfig.needToReplicate(TABLE_A)); 149 150 // Set table B 151 tableCfs = Maps.newHashMap(); 152 tableCfs.put(TABLE_B, null); 153 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 154 .setReplicateAllUserTables(false).setTableCFsMap(tableCfs).build(); 155 assertFalse(peerConfig.needToReplicate(TABLE_A)); 156 assertTrue(peerConfig.needToReplicate(TABLE_B)); 157 158 // 3. replication_all flag is false, and only config namespace in peer 159 // Set empty namespace set 160 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 161 .setReplicateAllUserTables(false).setNamespaces(Sets.newHashSet()).build(); 162 assertFalse(peerConfig.needToReplicate(TABLE_A)); 163 164 // Set namespace other 165 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 166 .setReplicateAllUserTables(false).setNamespaces(Sets.newHashSet(NAMESPACE_OTHER)).build(); 167 assertFalse(peerConfig.needToReplicate(TABLE_A)); 168 169 // Set namespace replication 170 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 171 .setReplicateAllUserTables(false).setNamespaces(Sets.newHashSet(NAMESPACE_REPLICATE)).build(); 172 assertTrue(peerConfig.needToReplicate(TABLE_A)); 173 174 // 4. replicate_all flag is false, and config namespaces and table-cfs both 175 // Namespaces config doesn't conflict with table-cfs config 176 tableCfs = Maps.newHashMap(); 177 tableCfs.put(TABLE_A, null); 178 peerConfig = 179 new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl().setReplicateAllUserTables(false) 180 .setTableCFsMap(tableCfs).setNamespaces(Sets.newHashSet(NAMESPACE_REPLICATE)).build(); 181 assertTrue(peerConfig.needToReplicate(TABLE_A)); 182 183 // Namespaces config conflicts with table-cfs config 184 tableCfs = Maps.newHashMap(); 185 tableCfs.put(TABLE_A, null); 186 peerConfig = 187 new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl().setReplicateAllUserTables(false) 188 .setTableCFsMap(tableCfs).setNamespaces(Sets.newHashSet(NAMESPACE_OTHER)).build(); 189 assertTrue(peerConfig.needToReplicate(TABLE_A)); 190 191 tableCfs = Maps.newHashMap(); 192 tableCfs.put(TABLE_B, null); 193 peerConfig = 194 new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl().setReplicateAllUserTables(false) 195 .setNamespaces(Sets.newHashSet(NAMESPACE_REPLICATE)).setTableCFsMap(tableCfs).build(); 196 assertTrue(peerConfig.needToReplicate(TABLE_A)); 197 } 198 199 @Test 200 public void testNeedToReplicateCFWithReplicatingAll() { 201 Map<TableName, List<String>> excludeTableCfs = Maps.newHashMap(); 202 excludeTableCfs.put(TABLE_A, null); 203 ReplicationPeerConfig peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 204 .setReplicateAllUserTables(true).setExcludeTableCFsMap(excludeTableCfs).build(); 205 assertFalse(peerConfig.needToReplicate(TABLE_A)); 206 assertFalse(peerConfig.needToReplicate(TABLE_A, FAMILY1)); 207 assertFalse(peerConfig.needToReplicate(TABLE_A, FAMILY2)); 208 209 excludeTableCfs = Maps.newHashMap(); 210 excludeTableCfs.put(TABLE_A, Lists.newArrayList()); 211 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 212 .setReplicateAllUserTables(true).setExcludeTableCFsMap(excludeTableCfs).build(); 213 assertFalse(peerConfig.needToReplicate(TABLE_A)); 214 assertFalse(peerConfig.needToReplicate(TABLE_A, FAMILY1)); 215 assertFalse(peerConfig.needToReplicate(TABLE_A, FAMILY2)); 216 217 excludeTableCfs = Maps.newHashMap(); 218 excludeTableCfs.put(TABLE_A, Lists.newArrayList(Bytes.toString(FAMILY1))); 219 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 220 .setReplicateAllUserTables(true).setExcludeTableCFsMap(excludeTableCfs).build(); 221 assertTrue(peerConfig.needToReplicate(TABLE_A)); 222 assertFalse(peerConfig.needToReplicate(TABLE_A, FAMILY1)); 223 assertTrue(peerConfig.needToReplicate(TABLE_A, FAMILY2)); 224 } 225 226 @Test 227 public void testNeedToReplicateCFWithoutReplicatingAll() { 228 Map<TableName, List<String>> tableCfs = Maps.newHashMap(); 229 tableCfs.put(TABLE_A, null); 230 ReplicationPeerConfig peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 231 .setReplicateAllUserTables(false).setTableCFsMap(tableCfs).build(); 232 assertTrue(peerConfig.needToReplicate(TABLE_A)); 233 assertTrue(peerConfig.needToReplicate(TABLE_A, FAMILY1)); 234 assertTrue(peerConfig.needToReplicate(TABLE_A, FAMILY2)); 235 236 tableCfs = Maps.newHashMap(); 237 tableCfs.put(TABLE_A, Lists.newArrayList()); 238 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 239 .setReplicateAllUserTables(false).setTableCFsMap(tableCfs).build(); 240 assertTrue(peerConfig.needToReplicate(TABLE_A)); 241 assertTrue(peerConfig.needToReplicate(TABLE_A, FAMILY1)); 242 assertTrue(peerConfig.needToReplicate(TABLE_A, FAMILY2)); 243 244 tableCfs = Maps.newHashMap(); 245 tableCfs.put(TABLE_A, Lists.newArrayList(Bytes.toString(FAMILY1))); 246 peerConfig = new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl() 247 .setReplicateAllUserTables(false).setTableCFsMap(tableCfs).build(); 248 assertTrue(peerConfig.needToReplicate(TABLE_A)); 249 assertTrue(peerConfig.needToReplicate(TABLE_A, FAMILY1)); 250 assertFalse(peerConfig.needToReplicate(TABLE_A, FAMILY2)); 251 } 252 253 @Test 254 public void testBaseReplicationPeerConfig() throws ReplicationException { 255 String customPeerConfigKey = "hbase.xxx.custom_config"; 256 String customPeerConfigValue = "test"; 257 String customPeerConfigUpdatedValue = "testUpdated"; 258 259 String customPeerConfigSecondKey = "hbase.xxx.custom_second_config"; 260 String customPeerConfigSecondValue = "testSecond"; 261 String customPeerConfigSecondUpdatedValue = "testSecondUpdated"; 262 263 ReplicationPeerConfig existingReplicationPeerConfig = getConfig(1); 264 265 // custom config not present 266 assertNull(existingReplicationPeerConfig.getConfiguration().get(customPeerConfigKey)); 267 268 Configuration conf = new Configuration(CONF); 269 conf.set(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG, 270 customPeerConfigKey.concat("=").concat(customPeerConfigValue).concat(";") 271 .concat(customPeerConfigSecondKey).concat("=").concat(customPeerConfigSecondValue)); 272 273 ReplicationPeerConfig updatedReplicationPeerConfig = ReplicationPeerConfigUtil 274 .updateReplicationBasePeerConfigs(conf, existingReplicationPeerConfig); 275 276 // validates base configs are present in replicationPeerConfig 277 assertEquals(customPeerConfigValue, 278 updatedReplicationPeerConfig.getConfiguration().get(customPeerConfigKey)); 279 assertEquals(customPeerConfigSecondValue, 280 updatedReplicationPeerConfig.getConfiguration().get(customPeerConfigSecondKey)); 281 282 // validates base configs get updated values even if config already present 283 conf.unset(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG); 284 conf.set(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG, 285 customPeerConfigKey.concat("=").concat(customPeerConfigUpdatedValue).concat(";") 286 .concat(customPeerConfigSecondKey).concat("=").concat(customPeerConfigSecondUpdatedValue)); 287 288 ReplicationPeerConfig replicationPeerConfigAfterValueUpdate = ReplicationPeerConfigUtil 289 .updateReplicationBasePeerConfigs(conf, updatedReplicationPeerConfig); 290 291 assertEquals(customPeerConfigUpdatedValue, 292 replicationPeerConfigAfterValueUpdate.getConfiguration().get(customPeerConfigKey)); 293 assertEquals(customPeerConfigSecondUpdatedValue, 294 replicationPeerConfigAfterValueUpdate.getConfiguration().get(customPeerConfigSecondKey)); 295 } 296 297 @Test 298 public void testBaseReplicationRemovePeerConfig() throws ReplicationException { 299 String customPeerConfigKey = "hbase.xxx.custom_config"; 300 String customPeerConfigValue = "test"; 301 ReplicationPeerConfig existingReplicationPeerConfig = getConfig(1); 302 303 // custom config not present 304 assertNull(existingReplicationPeerConfig.getConfiguration().get(customPeerConfigKey)); 305 306 Configuration conf = new Configuration(CONF); 307 conf.set(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG, 308 customPeerConfigKey.concat("=").concat(customPeerConfigValue)); 309 310 ReplicationPeerConfig updatedReplicationPeerConfig = ReplicationPeerConfigUtil 311 .updateReplicationBasePeerConfigs(conf, existingReplicationPeerConfig); 312 313 // validates base configs are present in replicationPeerConfig 314 assertEquals(customPeerConfigValue, 315 updatedReplicationPeerConfig.getConfiguration().get(customPeerConfigKey)); 316 317 conf.unset(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG); 318 conf.set(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG, 319 customPeerConfigKey.concat("=").concat("")); 320 321 ReplicationPeerConfig replicationPeerConfigRemoved = ReplicationPeerConfigUtil 322 .updateReplicationBasePeerConfigs(conf, updatedReplicationPeerConfig); 323 324 assertNull(replicationPeerConfigRemoved.getConfiguration().get(customPeerConfigKey)); 325 } 326 327 @Test 328 public void testBaseReplicationRemovePeerConfigWithNoExistingConfig() 329 throws ReplicationException { 330 String customPeerConfigKey = "hbase.xxx.custom_config"; 331 ReplicationPeerConfig existingReplicationPeerConfig = getConfig(1); 332 333 // custom config not present 334 assertNull(existingReplicationPeerConfig.getConfiguration().get(customPeerConfigKey)); 335 Configuration conf = new Configuration(CONF); 336 conf.set(ReplicationPeerConfigUtil.HBASE_REPLICATION_PEER_BASE_CONFIG, 337 customPeerConfigKey.concat("=").concat("")); 338 339 ReplicationPeerConfig updatedReplicationPeerConfig = ReplicationPeerConfigUtil 340 .updateReplicationBasePeerConfigs(conf, existingReplicationPeerConfig); 341 assertNull(updatedReplicationPeerConfig.getConfiguration().get(customPeerConfigKey)); 342 } 343}