1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
28 import com.google.common.collect.Lists;
29 import com.google.common.util.concurrent.AbstractService;
30
31 import static org.apache.hadoop.hbase.client.replication.ReplicationAdmin.REPLICATION_WALENTRYFILTER_CONFIG_KEY;
32
33
34
35
36
37
38 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
39 public abstract class BaseReplicationEndpoint extends AbstractService
40 implements ReplicationEndpoint {
41
42 private static final Log LOG = LogFactory.getLog(BaseReplicationEndpoint.class);
43 protected Context ctx;
44
45 @Override
46 public void init(Context context) throws IOException {
47 this.ctx = context;
48
49 if (this.ctx != null){
50 ReplicationPeer peer = this.ctx.getReplicationPeer();
51 if (peer != null){
52 peer.trackPeerConfigChanges(this);
53 } else {
54 LOG.warn("Not tracking replication peer config changes for Peer Id " + this.ctx.getPeerId() +
55 " because there's no such peer");
56 }
57 }
58 }
59
60 @Override
61
62
63
64 public void peerConfigUpdated(ReplicationPeerConfig rpc){
65
66 }
67
68
69 @Override
70 public WALEntryFilter getWALEntryfilter() {
71 ArrayList<WALEntryFilter> filters = Lists.newArrayList();
72 WALEntryFilter scopeFilter = getScopeWALEntryFilter();
73 if (scopeFilter != null) {
74 filters.add(scopeFilter);
75 }
76 WALEntryFilter tableCfFilter = getTableCfWALEntryFilter();
77 if (tableCfFilter != null) {
78 filters.add(tableCfFilter);
79 }
80 if (ctx != null && ctx.getPeerConfig() != null) {
81 String filterNameCSV = ctx.getPeerConfig().getConfiguration().get(REPLICATION_WALENTRYFILTER_CONFIG_KEY);
82 if (filterNameCSV != null && !filterNameCSV.isEmpty()) {
83 String[] filterNames = filterNameCSV.split(",");
84 for (String filterName : filterNames) {
85 try {
86 Class<?> clazz = Class.forName(filterName);
87 filters.add((WALEntryFilter) clazz.newInstance());
88 } catch (Exception e) {
89 LOG.error("Unable to create WALEntryFilter " + filterName, e);
90 }
91 }
92 }
93 }
94 return filters.isEmpty() ? null : new ChainWALEntryFilter(filters);
95 }
96
97
98
99 protected WALEntryFilter getScopeWALEntryFilter() {
100 return new ScopeWALEntryFilter();
101 }
102
103
104
105 protected WALEntryFilter getTableCfWALEntryFilter() {
106 return new TableCfWALEntryFilter(ctx.getReplicationPeer());
107 }
108
109 @Override
110 public boolean canReplicateToSameCluster() {
111 return false;
112 }
113
114 public void close(){
115 if(this.ctx != null) {
116 ReplicationPeer peer = this.ctx.getReplicationPeer();
117 peer.removeListenerOfPeerConfig(this);
118 }
119 }
120 }