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