View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * ReplicationEndpoint is a plugin which implements replication
37   * to other HBase clusters, or other systems. ReplicationEndpoint implementation
38   * can be specified at the peer creation time by specifying it
39   * in the {@link ReplicationPeerConfig}. A ReplicationEndpoint is run in a thread
40   * in each region server in the same process.
41   * <p>
42   * ReplicationEndpoint is closely tied to ReplicationSource in a producer-consumer
43   * relation. ReplicationSource is an HBase-private class which tails the logs and manages
44   * the queue of logs plus management and persistence of all the state for replication.
45   * ReplicationEndpoint on the other hand is responsible for doing the actual shipping
46   * and persisting of the WAL entries in the other cluster.
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    * Initialize the replication endpoint with the given context.
109    * @param context replication context
110    * @throws IOException
111    */
112   void init(Context context) throws IOException;
113 
114   /** Whether or not, the replication endpoint can replicate to it's source cluster with the same
115    * UUID */
116   boolean canReplicateToSameCluster();
117 
118   /**
119    * Returns a UUID of the provided peer id. Every HBase cluster instance has a persisted
120    * associated UUID. If the replication is not performed to an actual HBase cluster (but
121    * some other system), the UUID returned has to uniquely identify the connected target system.
122    * @return a UUID or null if the peer cluster does not exist or is not connected.
123    */
124   UUID getPeerUUID();
125 
126   /**
127    * Returns a WALEntryFilter to use for filtering out WALEntries from the log. Replication
128    * infrastructure will call this filter before sending the edits to shipEdits().
129    * @return a {@link WALEntryFilter} or null.
130    */
131   WALEntryFilter getWALEntryfilter();
132 
133   /**
134    * A context for {@link ReplicationEndpoint#replicate(ReplicateContext)} method.
135    */
136   @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
137   static class ReplicateContext {
138     List<Entry> entries;
139     int size;
140     @InterfaceAudience.Private
141     public ReplicateContext() {
142     }
143 
144     public ReplicateContext setEntries(List<Entry> entries) {
145       this.entries = entries;
146       return this;
147     }
148     public ReplicateContext setSize(int size) {
149       this.size = size;
150       return this;
151     }
152     public List<Entry> getEntries() {
153       return entries;
154     }
155     public int getSize() {
156       return size;
157     }
158   }
159 
160   /**
161    * Replicate the given set of entries (in the context) to the other cluster.
162    * Can block until all the given entries are replicated. Upon this method is returned,
163    * all entries that were passed in the context are assumed to be persisted in the
164    * target cluster.
165    * @param replicateContext a context where WAL entries and other
166    * parameters can be obtained.
167    */
168   boolean replicate(ReplicateContext replicateContext);
169 
170 }