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.List;
23 import java.util.UUID;
24
25 import org.apache.hadoop.hbase.Abortable;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.FileSystem;
29 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
30 import org.apache.hadoop.hbase.TableDescriptors;
31 import org.apache.hadoop.hbase.wal.WAL.Entry;
32 import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
33
34 import com.google.common.util.concurrent.Service;
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
50 public interface ReplicationEndpoint extends Service, ReplicationPeerConfigListener {
51
52 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
53 class Context {
54 private final Configuration localConf;
55 private final Configuration conf;
56 private final FileSystem fs;
57 private final TableDescriptors tableDescriptors;
58 private final ReplicationPeer replicationPeer;
59 private final String peerId;
60 private final UUID clusterId;
61 private final MetricsSource metrics;
62 private final Abortable abortable;
63
64 @InterfaceAudience.Private
65 public Context(
66 final Configuration localConf,
67 final Configuration conf,
68 final FileSystem fs,
69 final String peerId,
70 final UUID clusterId,
71 final ReplicationPeer replicationPeer,
72 final MetricsSource metrics,
73 final TableDescriptors tableDescriptors,
74 final Abortable abortable) {
75 this.localConf = localConf;
76 this.conf = conf;
77 this.fs = fs;
78 this.clusterId = clusterId;
79 this.peerId = peerId;
80 this.replicationPeer = replicationPeer;
81 this.metrics = metrics;
82 this.tableDescriptors = tableDescriptors;
83 this.abortable = abortable;
84 }
85 public Configuration getConfiguration() {
86 return conf;
87 }
88 public Configuration getLocalConfiguration() {
89 return localConf;
90 }
91 public FileSystem getFilesystem() {
92 return fs;
93 }
94 public UUID getClusterId() {
95 return clusterId;
96 }
97 public String getPeerId() {
98 return peerId;
99 }
100 public ReplicationPeerConfig getPeerConfig() {
101 return replicationPeer.getPeerConfig();
102 }
103 public ReplicationPeer getReplicationPeer() {
104 return replicationPeer;
105 }
106 public MetricsSource getMetrics() {
107 return metrics;
108 }
109 public TableDescriptors getTableDescriptors() {
110 return tableDescriptors;
111 }
112 public Abortable getAbortable() { return abortable; }
113 }
114
115
116
117
118
119
120 void init(Context context) throws IOException;
121
122
123
124 boolean canReplicateToSameCluster();
125
126
127
128
129
130
131
132 UUID getPeerUUID();
133
134
135
136
137
138
139 WALEntryFilter getWALEntryfilter();
140
141
142
143
144 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
145 static class ReplicateContext {
146 List<Entry> entries;
147 int size;
148 String walGroupId;
149 @InterfaceAudience.Private
150 public ReplicateContext() {
151 }
152
153 public ReplicateContext setEntries(List<Entry> entries) {
154 this.entries = entries;
155 return this;
156 }
157 public ReplicateContext setSize(int size) {
158 this.size = size;
159 return this;
160 }
161 public ReplicateContext setWalGroupId(String walGroupId) {
162 this.walGroupId = walGroupId;
163 return this;
164 }
165 public List<Entry> getEntries() {
166 return entries;
167 }
168 public int getSize() {
169 return size;
170 }
171 public String getWalGroupId(){
172 return walGroupId;
173 }
174 }
175
176
177
178
179
180
181
182
183
184 boolean replicate(ReplicateContext replicateContext);
185
186 }