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.regionserver.wal.HLog;
30  import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
31  
32  import com.google.common.util.concurrent.Service;
33  
34  /**
35   * ReplicationEndpoint is a plugin which implements replication
36   * to other HBase clusters, or other systems. ReplicationEndpoint implementation
37   * can be specified at the peer creation time by specifying it
38   * in the {@link ReplicationPeerConfig}. A ReplicationEndpoint is run in a thread
39   * in each region server in the same process.
40   * <p>
41   * ReplicationEndpoint is closely tied to ReplicationSource in a producer-consumer
42   * relation. ReplicationSource is an HBase-private class which tails the logs and manages
43   * the queue of logs plus management and persistence of all the state for replication.
44   * ReplicationEndpoint on the other hand is responsible for doing the actual shipping
45   * and persisting of the WAL entries in the other cluster.
46   */
47  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
48  public interface ReplicationEndpoint extends Service, ReplicationPeerConfigListener {
49  
50    @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
51    class Context {
52      private final Configuration conf;
53      private final FileSystem fs;
54      private final ReplicationPeer replicationPeer;
55      private final String peerId;
56      private final UUID clusterId;
57      private final MetricsSource metrics;
58  
59      @InterfaceAudience.Private
60      public Context(
61          final Configuration conf,
62          final FileSystem fs,
63          final String peerId,
64          final UUID clusterId,
65          final ReplicationPeer replicationPeer,
66          final MetricsSource metrics) {
67        this.conf = conf;
68        this.fs = fs;
69        this.clusterId = clusterId;
70        this.peerId = peerId;
71        this.replicationPeer = replicationPeer;
72        this.metrics = metrics;
73      }
74      public Configuration getConfiguration() {
75        return conf;
76      }
77      public FileSystem getFilesystem() {
78        return fs;
79      }
80      public UUID getClusterId() {
81        return clusterId;
82      }
83      public String getPeerId() {
84        return peerId;
85      }
86      public ReplicationPeerConfig getPeerConfig() {
87        return replicationPeer.getPeerConfig();
88      }
89      public ReplicationPeer getReplicationPeer() {
90        return replicationPeer;
91      }
92      public MetricsSource getMetrics() {
93        return metrics;
94      }
95    }
96  
97    /**
98     * Initialize the replication endpoint with the given context.
99     * @param context replication context
100    * @throws IOException
101    */
102   void init(Context context) throws IOException;
103 
104   /** Whether or not, the replication endpoint can replicate to it's source cluster with the same
105    * UUID */
106   boolean canReplicateToSameCluster();
107 
108   /**
109    * Returns a UUID of the provided peer id. Every HBase cluster instance has a persisted
110    * associated UUID. If the replication is not performed to an actual HBase cluster (but
111    * some other system), the UUID returned has to uniquely identify the connected target system.
112    * @return a UUID or null if the peer cluster does not exist or is not connected.
113    */
114   UUID getPeerUUID();
115 
116   /**
117    * Returns a WALEntryFilter to use for filtering out WALEntries from the log. Replication
118    * infrastructure will call this filter before sending the edits to shipEdits().
119    * @return a {@link WALEntryFilter} or null.
120    */
121   WALEntryFilter getWALEntryfilter();
122 
123   /**
124    * A context for {@link ReplicationEndpoint#replicate(ReplicateContext)} method.
125    */
126   @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
127   static class ReplicateContext {
128     List<HLog.Entry> entries;
129     int size;
130     @InterfaceAudience.Private
131     public ReplicateContext() {
132     }
133 
134     public ReplicateContext setEntries(List<HLog.Entry> entries) {
135       this.entries = entries;
136       return this;
137     }
138     public ReplicateContext setSize(int size) {
139       this.size = size;
140       return this;
141     }
142     public List<HLog.Entry> getEntries() {
143       return entries;
144     }
145     public int getSize() {
146       return size;
147     }
148   }
149 
150   /**
151    * Replicate the given set of entries (in the context) to the other cluster.
152    * Can block until all the given entries are replicated. Upon this method is returned,
153    * all entries that were passed in the context are assumed to be persisted in the
154    * target cluster.
155    * @param replicateContext a context where WAL entries and other
156    * parameters can be obtained.
157    */
158   boolean replicate(ReplicateContext replicateContext);
159 
160 }