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