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.regionserver;
20  
21  import java.io.IOException;
22  import java.net.ConnectException;
23  import java.net.SocketTimeoutException;
24  import java.net.UnknownHostException;
25  import java.util.ArrayList;
26  import java.util.Collections;
27  import java.util.HashMap;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.concurrent.Callable;
31  import java.util.concurrent.CompletionService;
32  import java.util.concurrent.ExecutionException;
33  import java.util.concurrent.ExecutorCompletionService;
34  import java.util.concurrent.Future;
35  import java.util.concurrent.LinkedBlockingQueue;
36  import java.util.concurrent.ThreadPoolExecutor;
37  import java.util.concurrent.TimeUnit;
38  import java.util.regex.Matcher;
39  import java.util.regex.Pattern;
40  
41  import org.apache.commons.lang.StringUtils;
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.conf.Configuration;
45  import org.apache.hadoop.fs.Path;
46  import org.apache.hadoop.hbase.Abortable;
47  import org.apache.hadoop.hbase.HBaseConfiguration;
48  import org.apache.hadoop.hbase.HConstants;
49  import org.apache.hadoop.hbase.TableName;
50  import org.apache.hadoop.hbase.TableNotFoundException;
51  import org.apache.hadoop.hbase.classification.InterfaceAudience;
52  import org.apache.hadoop.hbase.client.Connection;
53  import org.apache.hadoop.hbase.client.ConnectionFactory;
54  import org.apache.hadoop.hbase.client.HConnection;
55  import org.apache.hadoop.hbase.client.HConnectionManager;
56  import org.apache.hadoop.hbase.ipc.RpcServer;
57  import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
58  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
59  import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
60  import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
61  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
62  import org.apache.hadoop.hbase.util.Bytes;
63  import org.apache.hadoop.hbase.util.FSUtils;
64  import org.apache.hadoop.hbase.wal.WAL.Entry;
65  import org.apache.hadoop.ipc.RemoteException;
66  
67  /**
68   * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} 
69   * implementation for replicating to another HBase cluster.
70   * For the slave cluster it selects a random number of peers
71   * using a replication ratio. For example, if replication ration = 0.1
72   * and slave cluster has 100 region servers, 10 will be selected.
73   * <p>
74   * A stream is considered down when we cannot contact a region server on the
75   * peer cluster for more than 55 seconds by default.
76   * </p>
77   */
78  @InterfaceAudience.Private
79  public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoint {
80  
81    private static final Log LOG = LogFactory.getLog(HBaseInterClusterReplicationEndpoint.class);
82  
83    private static final long DEFAULT_MAX_TERMINATION_WAIT_MULTIPLIER = 2;
84  
85    private HConnection conn;
86    private Configuration localConf;
87    private Configuration conf;
88    // How long should we sleep for each retry
89    private long sleepForRetries;
90    // Maximum number of retries before taking bold actions
91    private int maxRetriesMultiplier;
92    // Socket timeouts require even bolder actions since we don't want to DDOS
93    private int socketTimeoutMultiplier;
94    // Amount of time for shutdown to wait for all tasks to complete
95    private long maxTerminationWait;
96    // Size limit for replication RPCs, in bytes
97    private int replicationRpcLimit;
98    //Metrics for this source
99    private MetricsSource metrics;
100   // Handles connecting to peer region servers
101   private ReplicationSinkManager replicationSinkMgr;
102   private boolean peersSelected = false;
103   private String replicationClusterId = "";
104   private ThreadPoolExecutor exec;
105   private int maxThreads;
106   private Path baseNamespaceDir;
107   private Path hfileArchiveDir;
108   private boolean replicationBulkLoadDataEnabled;
109   private Abortable abortable;
110   private boolean dropOnDeletedTables;
111 
112   @Override
113   public void init(Context context) throws IOException {
114     super.init(context);
115     this.conf = HBaseConfiguration.create(ctx.getConfiguration());
116     this.localConf = HBaseConfiguration.create(ctx.getLocalConfiguration());
117     decorateConf();
118     this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 300);
119     this.socketTimeoutMultiplier = this.conf.getInt("replication.source.socketTimeoutMultiplier",
120         maxRetriesMultiplier);
121     // A Replicator job is bound by the RPC timeout. We will wait this long for all Replicator
122     // tasks to terminate when doStop() is called.
123     long maxTerminationWaitMultiplier = this.conf.getLong(
124         "replication.source.maxterminationmultiplier",
125         DEFAULT_MAX_TERMINATION_WAIT_MULTIPLIER);
126     this.maxTerminationWait = maxTerminationWaitMultiplier *
127         this.conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
128     // TODO: This connection is replication specific or we should make it particular to
129     // replication and make replication specific settings such as compression or codec to use
130     // passing Cells.
131     this.conn = HConnectionManager.createConnection(this.conf);
132     this.sleepForRetries =
133         this.conf.getLong("replication.source.sleepforretries", 1000);
134     this.metrics = context.getMetrics();
135     // ReplicationQueueInfo parses the peerId out of the znode for us
136     this.replicationSinkMgr = new ReplicationSinkManager(conn, ctx.getPeerId(), this, this.conf);
137     // per sink thread pool
138     this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
139       HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
140     this.exec = new ThreadPoolExecutor(maxThreads, maxThreads, 60, TimeUnit.SECONDS,
141         new LinkedBlockingQueue<Runnable>());
142     this.exec.allowCoreThreadTimeOut(true);
143     this.abortable = ctx.getAbortable();
144     // Set the size limit for replication RPCs to 95% of the max request size.
145     // We could do with less slop if we have an accurate estimate of encoded size. Being
146     // conservative for now.
147     this.replicationRpcLimit = (int)(0.95 * (double)conf.getLong(RpcServer.MAX_REQUEST_SIZE,
148       RpcServer.DEFAULT_MAX_REQUEST_SIZE));
149     this.dropOnDeletedTables =
150         this.conf.getBoolean(HConstants.REPLICATION_DROP_ON_DELETED_TABLE_KEY, false);
151 
152     this.replicationBulkLoadDataEnabled =
153         conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
154           HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
155     if (this.replicationBulkLoadDataEnabled) {
156       replicationClusterId = this.conf.get(HConstants.REPLICATION_CLUSTER_ID);
157     }
158     // Construct base namespace directory and hfile archive directory path
159     Path rootDir = FSUtils.getRootDir(conf);
160     Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR);
161     baseNamespaceDir = new Path(rootDir, baseNSDir);
162     hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir));
163   }
164 
165   private void decorateConf() {
166     String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
167     if (StringUtils.isNotEmpty(replicationCodec)) {
168       this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
169     }
170   }
171 
172   private void connectToPeers() {
173     getRegionServers();
174 
175     int sleepMultiplier = 1;
176 
177     // Connect to peer cluster first, unless we have to stop
178     while (this.isRunning() && replicationSinkMgr.getNumSinks() == 0) {
179       replicationSinkMgr.chooseSinks();
180       if (this.isRunning() && replicationSinkMgr.getNumSinks() == 0) {
181         if (sleepForRetries("Waiting for peers", sleepMultiplier)) {
182           sleepMultiplier++;
183         }
184       }
185     }
186   }
187 
188   /**
189    * Do the sleeping logic
190    * @param msg Why we sleep
191    * @param sleepMultiplier by how many times the default sleeping time is augmented
192    * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
193    */
194   protected boolean sleepForRetries(String msg, int sleepMultiplier) {
195     try {
196       if (LOG.isTraceEnabled()) {
197         LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
198       }
199       Thread.sleep(this.sleepForRetries * sleepMultiplier);
200     } catch (InterruptedException e) {
201       LOG.debug("Interrupted while sleeping between retries");
202     }
203     return sleepMultiplier < maxRetriesMultiplier;
204   }
205 
206   private List<List<Entry>> createBatches(final List<Entry> entries) {
207     int numSinks = Math.max(replicationSinkMgr.getNumSinks(), 1);
208     int n = Math.min(Math.min(this.maxThreads, entries.size()/100+1), numSinks);
209     // Maintains the current batch for a given partition index
210     Map<Integer, List<Entry>> entryMap = new HashMap<>(n);
211     List<List<Entry>> entryLists = new ArrayList<>();
212     int[] sizes = new int[n];
213 
214     for (int i = 0; i < n; i++) {
215       entryMap.put(i, new ArrayList<Entry>(entries.size()/n+1));
216     }
217 
218     for (Entry e: entries) {
219       int index = Math.abs(Bytes.hashCode(e.getKey().getEncodedRegionName())%n);
220       int entrySize = (int)e.getKey().estimatedSerializedSizeOf() +
221           (int)e.getEdit().estimatedSerializedSizeOf();
222       // If this batch is oversized, add it to final list and initialize a new empty batch
223       if (sizes[index] > 0 /* must include at least one entry */ &&
224           sizes[index] + entrySize > replicationRpcLimit) {
225         entryLists.add(entryMap.get(index));
226         entryMap.put(index, new ArrayList<Entry>());
227         sizes[index] = 0;
228       }
229       entryMap.get(index).add(e);
230       sizes[index] += entrySize;
231     }
232 
233     entryLists.addAll(entryMap.values());
234     return entryLists;
235   }
236 
237   private TableName parseTable(String msg) {
238     // ... TableNotFoundException: '<table>'/n...
239     Pattern p = Pattern.compile("TableNotFoundException: \\'([\\S]*)\\'");
240     Matcher m = p.matcher(msg);
241     if (m.find()) {
242       String table = m.group(1);
243       try {
244         // double check that table is a valid table name
245         TableName.valueOf(TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(table)));
246         return TableName.valueOf(table);
247       } catch (IllegalArgumentException ignore) {
248       }
249     }
250     return null;
251   }
252 
253   // Filter a set of batches by TableName
254   private List<List<Entry>> filterBatches(final List<List<Entry>> oldEntryList, TableName table) {
255     List<List<Entry>> entryLists = new ArrayList<>();
256     for (List<Entry> entries : oldEntryList) {
257       ArrayList<Entry> thisList = new ArrayList<Entry>(entries.size());
258       entryLists.add(thisList);
259       for (Entry e : entries) {
260         if (!e.getKey().getTablename().equals(table)) {
261           thisList.add(e);
262         }
263       }
264     }
265     return entryLists;
266   }
267 
268   private void reconnectToPeerCluster() {
269     HConnection connection = null;
270     try {
271       connection = HConnectionManager.createConnection(this.conf);
272     } catch (IOException ioe) {
273       LOG.warn("Failed to create connection for peer cluster", ioe);
274     }
275     if (connection != null) {
276       this.conn = connection;
277     }
278   }
279 
280   /**
281    * Do the shipping logic
282    */
283   @Override
284   public boolean replicate(ReplicateContext replicateContext) {
285     CompletionService<Integer> pool = new ExecutorCompletionService<Integer>(this.exec);
286     List<List<Entry>> batches;
287     String walGroupId = replicateContext.getWalGroupId();
288     int sleepMultiplier = 1;
289 
290     if (!peersSelected && this.isRunning()) {
291       connectToPeers();
292       peersSelected = true;
293     }
294 
295     int numSinks = replicationSinkMgr.getNumSinks();
296     if (numSinks == 0) {
297       LOG.warn("No replication sinks found, returning without replicating. The source should retry"
298           + " with the same set of edits.");
299       return false;
300     }
301 
302     batches = createBatches(replicateContext.getEntries());
303 
304     while (this.isRunning() && !exec.isShutdown()) {
305       if (!isPeerEnabled()) {
306         if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
307           sleepMultiplier++;
308         }
309         continue;
310       }
311       if (this.conn == null || this.conn.isClosed()) {
312         reconnectToPeerCluster();
313       }
314       try {
315         int futures = 0;
316         for (int i=0; i<batches.size(); i++) {
317           List<Entry> entries = batches.get(i);
318           if (!entries.isEmpty()) {
319             if (LOG.isTraceEnabled()) {
320               LOG.trace("Submitting " + entries.size() +
321                   " entries of total size " + replicateContext.getSize());
322             }
323             // RuntimeExceptions encountered here bubble up and are handled in ReplicationSource
324             pool.submit(createReplicator(entries, i));
325             futures++;
326           }
327         }
328         IOException iox = null;
329 
330         long lastWriteTime = 0;
331         for (int i=0; i<futures; i++) {
332           try {
333             // wait for all futures, remove successful parts
334             // (only the remaining parts will be retried)
335             Future<Integer> f = pool.take();
336             int index = f.get().intValue();
337             List<Entry> batch = batches.get(index);
338             batches.set(index, Collections.<Entry>emptyList()); // remove successful batch
339             // Find the most recent write time in the batch
340             long writeTime = batch.get(batch.size() - 1).getKey().getWriteTime();
341             if (writeTime > lastWriteTime) {
342               lastWriteTime = writeTime;
343             }
344           } catch (InterruptedException ie) {
345             iox =  new IOException(ie);
346           } catch (ExecutionException ee) {
347             // cause must be an IOException
348             iox = (IOException)ee.getCause();
349           }
350         }
351         if (iox != null) {
352           // if we had any exceptions, try again
353           throw iox;
354         }
355         // update metrics
356         if (lastWriteTime > 0) {
357           this.metrics.setAgeOfLastShippedOp(lastWriteTime, walGroupId);
358         }
359         return true;
360 
361       } catch (IOException ioe) {
362         // Didn't ship anything, but must still age the last time we did
363         this.metrics.refreshAgeOfLastShippedOp(walGroupId);
364         if (ioe instanceof RemoteException) {
365           ioe = ((RemoteException) ioe).unwrapRemoteException();
366           LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);
367           if (ioe instanceof TableNotFoundException) {
368             if (dropOnDeletedTables) {
369               // this is a bit fragile, but cannot change how TNFE is serialized
370               // at least check whether the table name is legal
371               TableName table = parseTable(ioe.getMessage());
372               if (table != null) {
373                 try (Connection localConn =
374                     ConnectionFactory.createConnection(ctx.getLocalConfiguration())) {
375                   if (!localConn.getAdmin().tableExists(table)) {
376                     // Would potentially be better to retry in one of the outer loops
377                     // and add a table filter there; but that would break the encapsulation,
378                     // so we're doing the filtering here.
379                     LOG.info("Missing table detected at sink, local table also does not exist, filtering edits for '"+table+"'");
380                     batches = filterBatches(batches, table);
381                     continue;
382                   }
383                 } catch (IOException iox) {
384                   LOG.warn("Exception checking for local table: ", iox);
385                 }
386               }
387             }
388             // fall through and sleep below
389           } else {
390             LOG.warn("Peer encountered RemoteException, rechecking all sinks: ", ioe);
391             replicationSinkMgr.chooseSinks();
392           }
393         } else {
394           if (ioe instanceof SocketTimeoutException) {
395             // This exception means we waited for more than 60s and nothing
396             // happened, the cluster is alive and calling it right away
397             // even for a test just makes things worse.
398             sleepForRetries("Encountered a SocketTimeoutException. Since the " +
399               "call to the remote cluster timed out, which is usually " +
400               "caused by a machine failure or a massive slowdown",
401               this.socketTimeoutMultiplier);
402           } else if (ioe instanceof ConnectException || ioe instanceof UnknownHostException) {
403             LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
404             replicationSinkMgr.chooseSinks();
405           } else {
406             LOG.warn("Can't replicate because of a local or network error: ", ioe);
407           }
408         }
409         if (sleepForRetries("Since we are unable to replicate", sleepMultiplier)) {
410           sleepMultiplier++;
411         }
412       }
413     }
414     return false; // in case we exited before replicating
415   }
416 
417   protected boolean isPeerEnabled() {
418     return ctx.getReplicationPeer().getPeerState() == PeerState.ENABLED;
419   }
420 
421   @Override
422   protected void doStop() {
423     disconnect(); //don't call super.doStop()
424     if (this.conn != null) {
425       try {
426         this.conn.close();
427         this.conn = null;
428       } catch (IOException e) {
429         LOG.warn("Failed to close the connection");
430       }
431     }
432     // Allow currently running replication tasks to finish
433     exec.shutdown();
434     try {
435       exec.awaitTermination(maxTerminationWait, TimeUnit.MILLISECONDS);
436     } catch (InterruptedException e) {
437     }
438     // Abort if the tasks did not terminate in time
439     if (!exec.isTerminated()) {
440       String errMsg = "HBaseInterClusterReplicationEndpoint termination failed. The " +
441           "ThreadPoolExecutor failed to finish all tasks within " + maxTerminationWait + "ms. " +
442           "Aborting to prevent Replication from deadlocking. See HBASE-16081.";
443       abortable.abort(errMsg, new IOException(errMsg));
444     }
445     close();
446     notifyStopped();
447   }
448 
449   // is this needed? Nobody else will call doStop() otherwise
450   @Override
451   public State stopAndWait() {
452     doStop();
453     return super.stopAndWait();
454   }
455 
456   protected Replicator createReplicator(List<Entry> entries, int ordinal) {
457     return new Replicator(entries, ordinal);
458   }
459 
460   protected class Replicator implements Callable<Integer> {
461     private List<Entry> entries;
462     private int ordinal;
463     public Replicator(List<Entry> entries, int ordinal) {
464       this.entries = entries;
465       this.ordinal = ordinal;
466     }
467 
468     protected void replicateEntries(BlockingInterface rrs, final List<Entry> batch,
469         String replicationClusterId, Path baseNamespaceDir, Path hfileArchiveDir)
470         throws IOException {
471       if (LOG.isTraceEnabled()) {
472         long size = 0;
473         for (Entry e: entries) {
474           size += e.getKey().estimatedSerializedSizeOf();
475           size += e.getEdit().estimatedSerializedSizeOf();
476         }
477         LOG.trace("Replicating batch " + System.identityHashCode(entries) + " of " +
478             entries.size() + " entries with total size " + size + " bytes to " +
479             replicationClusterId);
480       }
481       try {
482         ReplicationProtbufUtil.replicateWALEntry(rrs, batch.toArray(new Entry[batch.size()]),
483           replicationClusterId, baseNamespaceDir, hfileArchiveDir);
484         if (LOG.isTraceEnabled()) {
485           LOG.trace("Completed replicating batch " + System.identityHashCode(entries));
486         }
487       } catch (IOException e) {
488         if (LOG.isTraceEnabled()) {
489           LOG.trace("Failed replicating batch " + System.identityHashCode(entries), e);
490         }
491         throw e;
492       }
493     }
494 
495     @Override
496     public Integer call() throws IOException {
497       SinkPeer sinkPeer = null;
498       try {
499         sinkPeer = replicationSinkMgr.getReplicationSink();
500         BlockingInterface rrs = sinkPeer.getRegionServer();
501         replicateEntries(rrs, entries, replicationClusterId, baseNamespaceDir, hfileArchiveDir);
502         replicationSinkMgr.reportSinkSuccess(sinkPeer);
503         return ordinal;
504       } catch (IOException ioe) {
505         if (sinkPeer != null) {
506           replicationSinkMgr.reportBadSink(sinkPeer);
507         }
508         throw ioe;
509       }
510     }
511   }
512 }