Skip to content

Commit

Permalink
HBASE-23345 Table need to replication unless all of cfs are excluded (a…
Browse files Browse the repository at this point in the history
…pache#881)

Signed-off-by: stack <[email protected]>
Signed-off-by: Guanghao Zhang <[email protected]>
  • Loading branch information
ddupg authored and infraio committed Dec 3, 2019
1 parent 97e0107 commit 27cfe1b
Show file tree
Hide file tree
Showing 11 changed files with 278 additions and 363 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -392,22 +392,31 @@ public String toString() {
* @return true if the table need replicate to the peer cluster
*/
public boolean needToReplicate(TableName table) {
String namespace = table.getNamespaceAsString();
if (replicateAllUserTables) {
if (excludeNamespaces != null && excludeNamespaces.contains(table.getNamespaceAsString())) {
// replicate all user tables, but filter by exclude namespaces and table-cfs config
if (excludeNamespaces != null && excludeNamespaces.contains(namespace)) {
return false;
}
if (excludeTableCFsMap != null && excludeTableCFsMap.containsKey(table)) {
return false;
// trap here, must check existence first since HashMap allows null value.
if (excludeTableCFsMap == null || !excludeTableCFsMap.containsKey(table)) {
return true;
}
return true;
Collection<String> cfs = excludeTableCFsMap.get(table);
// if cfs is null or empty then we can make sure that we do not need to replicate this table,
// otherwise, we may still need to replicate the table but filter out some families.
return cfs != null && !cfs.isEmpty();
} else {
if (namespaces != null && namespaces.contains(table.getNamespaceAsString())) {
return true;
// Not replicate all user tables, so filter by namespaces and table-cfs config
if (namespaces == null && tableCFsMap == null) {
return false;
}
if (tableCFsMap != null && tableCFsMap.containsKey(table)) {
// First filter by namespaces config
// If table's namespace in peer config, all the tables data are applicable for replication
if (namespaces != null && namespaces.contains(namespace)) {
return true;
}
return false;
return tableCFsMap != null && tableCFsMap.containsKey(table);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,17 @@
*/
package org.apache.hadoop.hbase.replication;

import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.BuilderStyleTest;
import org.junit.Assert;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
Expand All @@ -32,6 +39,9 @@ public class TestReplicationPeerConfig {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestReplicationPeerConfig.class);

private static TableName TABLE_A = TableName.valueOf("replication", "testA");
private static TableName TABLE_B = TableName.valueOf("replication", "testB");

@Test
public void testClassMethodsAreBuilderStyle() {
/* ReplicationPeerConfig should have a builder style setup where setXXX/addXXX methods
Expand All @@ -48,4 +58,196 @@ public void testClassMethodsAreBuilderStyle() {

BuilderStyleTest.assertClassesAreBuilderStyle(ReplicationPeerConfig.class);
}

@Test
public void testNeedToReplicateWithReplicatingAll() {
ReplicationPeerConfig peerConfig;
ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl builder =
new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl();
Map<TableName, List<String>> tableCfs = new HashMap<>();
Set<String> namespaces = new HashSet<>();

// 1. replication_all flag is true, no namespaces and table-cfs config
builder.setReplicateAllUserTables(true);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// 2. replicate_all flag is true, and config in excludedTableCfs
builder.setExcludeNamespaces(null);
// empty map
tableCfs = new HashMap<>();
builder.setReplicateAllUserTables(true);
builder.setExcludeTableCFsMap(tableCfs);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// table testB
tableCfs = new HashMap<>();
tableCfs.put(TABLE_B, null);
builder.setReplicateAllUserTables(true);
builder.setExcludeTableCFsMap(tableCfs);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// table testA
tableCfs = new HashMap<>();
tableCfs.put(TABLE_A, null);
builder.setReplicateAllUserTables(true);
builder.setExcludeTableCFsMap(tableCfs);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// 3. replicate_all flag is true, and config in excludeNamespaces
builder.setExcludeTableCFsMap(null);
// empty set
namespaces = new HashSet<>();
builder.setReplicateAllUserTables(true);
builder.setExcludeNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// namespace default
namespaces = new HashSet<>();
namespaces.add("default");
builder.setReplicateAllUserTables(true);
builder.setExcludeNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// namespace replication
namespaces = new HashSet<>();
namespaces.add("replication");
builder.setReplicateAllUserTables(true);
builder.setExcludeNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// 4. replicate_all flag is true, and config excludeNamespaces and excludedTableCfs both
// Namespaces config doesn't conflict with table-cfs config
namespaces = new HashSet<>();
tableCfs = new HashMap<>();
namespaces.add("replication");
tableCfs.put(TABLE_A, null);
builder.setReplicateAllUserTables(true);
builder.setExcludeTableCFsMap(tableCfs);
builder.setExcludeNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// Namespaces config conflicts with table-cfs config
namespaces = new HashSet<>();
tableCfs = new HashMap<>();
namespaces.add("default");
tableCfs.put(TABLE_A, null);
builder.setReplicateAllUserTables(true);
builder.setExcludeTableCFsMap(tableCfs);
builder.setExcludeNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

namespaces = new HashSet<>();
tableCfs = new HashMap<>();
namespaces.add("replication");
tableCfs.put(TABLE_B, null);
builder.setReplicateAllUserTables(true);
builder.setExcludeTableCFsMap(tableCfs);
builder.setExcludeNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

}

@Test
public void testNeedToReplicateWithoutReplicatingAll() {
ReplicationPeerConfig peerConfig;
ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl builder =
new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl();
Map<TableName, List<String>> tableCfs = new HashMap<>();
Set<String> namespaces = new HashSet<>();

// 1. replication_all flag is false, no namespaces and table-cfs config
builder.setReplicateAllUserTables(false);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// 2. replicate_all flag is false, and only config table-cfs in peer
// empty map
builder.setReplicateAllUserTables(false);
builder.setTableCFsMap(tableCfs);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// table testB
tableCfs = new HashMap<>();
tableCfs.put(TABLE_B, null);
builder.setReplicateAllUserTables(false);
builder.setTableCFsMap(tableCfs);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// table testA
tableCfs = new HashMap<>();
tableCfs.put(TABLE_A, null);
builder.setReplicateAllUserTables(false);
builder.setTableCFsMap(tableCfs);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// 3. replication_all flag is false, and only config namespace in peer
builder.setTableCFsMap(null);
// empty set
builder.setReplicateAllUserTables(false);
builder.setNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// namespace default
namespaces = new HashSet<>();
namespaces.add("default");
builder.setReplicateAllUserTables(false);
builder.setNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertFalse(peerConfig.needToReplicate(TABLE_A));

// namespace replication
namespaces = new HashSet<>();
namespaces.add("replication");
builder.setReplicateAllUserTables(false);
builder.setNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// 4. replicate_all flag is false, and config namespaces and table-cfs both
// Namespaces config doesn't conflict with table-cfs config
namespaces = new HashSet<>();
tableCfs = new HashMap<>();
namespaces.add("replication");
tableCfs.put(TABLE_A, null);
builder.setReplicateAllUserTables(false);
builder.setTableCFsMap(tableCfs);
builder.setNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

// Namespaces config conflicts with table-cfs config
namespaces = new HashSet<>();
tableCfs = new HashMap<>();
namespaces.add("default");
tableCfs.put(TABLE_A, null);
builder.setReplicateAllUserTables(false);
builder.setTableCFsMap(tableCfs);
builder.setNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));

namespaces = new HashSet<>();
tableCfs = new HashMap<>();
namespaces.add("replication");
tableCfs.put(TABLE_B, null);
builder.setReplicateAllUserTables(false);
builder.setTableCFsMap(tableCfs);
builder.setNamespaces(namespaces);
peerConfig = builder.build();
Assert.assertTrue(peerConfig.needToReplicate(TABLE_A));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -154,44 +154,6 @@ public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c)
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
}

/**
* Returns whether we should replicate the given table.
*/
public static boolean contains(ReplicationPeerConfig peerConfig, TableName tableName) {
String namespace = tableName.getNamespaceAsString();
if (peerConfig.replicateAllUserTables()) {
// replicate all user tables, but filter by exclude namespaces and table-cfs config
Set<String> excludeNamespaces = peerConfig.getExcludeNamespaces();
if (excludeNamespaces != null && excludeNamespaces.contains(namespace)) {
return false;
}
Map<TableName, List<String>> excludedTableCFs = peerConfig.getExcludeTableCFsMap();
// trap here, must check existence first since HashMap allows null value.
if (excludedTableCFs == null || !excludedTableCFs.containsKey(tableName)) {
return true;
}
List<String> cfs = excludedTableCFs.get(tableName);
// if cfs is null or empty then we can make sure that we do not need to replicate this table,
// otherwise, we may still need to replicate the table but filter out some families.
return cfs != null && !cfs.isEmpty();
} else {
// Not replicate all user tables, so filter by namespaces and table-cfs config
Set<String> namespaces = peerConfig.getNamespaces();
Map<TableName, List<String>> tableCFs = peerConfig.getTableCFsMap();

if (namespaces == null && tableCFs == null) {
return false;
}

// First filter by namespaces config
// If table's namespace in peer config, all the tables data are applicable for replication
if (namespaces != null && namespaces.contains(namespace)) {
return true;
}
return tableCFs != null && tableCFs.containsKey(tableName);
}
}

public static FileSystem getRemoteWALFileSystem(Configuration conf, String remoteWALDir)
throws IOException {
return new Path(remoteWALDir).getFileSystem(conf);
Expand Down
Loading

0 comments on commit 27cfe1b

Please sign in to comment.