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