View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.wal;
20  
21  import java.io.EOFException;
22  import java.io.FileNotFoundException;
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.text.ParseException;
26  import java.util.ArrayList;
27  import java.util.Collections;
28  import java.util.HashMap;
29  import java.util.HashSet;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.NavigableSet;
34  import java.util.Set;
35  import java.util.TreeMap;
36  import java.util.TreeSet;
37  import java.util.UUID;
38  import java.util.concurrent.Callable;
39  import java.util.concurrent.CompletionService;
40  import java.util.concurrent.ConcurrentHashMap;
41  import java.util.concurrent.ExecutionException;
42  import java.util.concurrent.ExecutorCompletionService;
43  import java.util.concurrent.Future;
44  import java.util.concurrent.ThreadFactory;
45  import java.util.concurrent.ThreadPoolExecutor;
46  import java.util.concurrent.TimeUnit;
47  import java.util.concurrent.atomic.AtomicBoolean;
48  import java.util.concurrent.atomic.AtomicLong;
49  import java.util.concurrent.atomic.AtomicReference;
50  import java.util.regex.Matcher;
51  import java.util.regex.Pattern;
52  
53  import org.apache.commons.logging.Log;
54  import org.apache.commons.logging.LogFactory;
55  import org.apache.hadoop.conf.Configuration;
56  import org.apache.hadoop.fs.FileAlreadyExistsException;
57  import org.apache.hadoop.fs.FileStatus;
58  import org.apache.hadoop.fs.FileSystem;
59  import org.apache.hadoop.fs.Path;
60  import org.apache.hadoop.fs.PathFilter;
61  import org.apache.hadoop.hbase.Cell;
62  import org.apache.hadoop.hbase.CellScanner;
63  import org.apache.hadoop.hbase.CellUtil;
64  import org.apache.hadoop.hbase.CoordinatedStateException;
65  import org.apache.hadoop.hbase.CoordinatedStateManager;
66  import org.apache.hadoop.hbase.HBaseConfiguration;
67  import org.apache.hadoop.hbase.HConstants;
68  import org.apache.hadoop.hbase.HRegionInfo;
69  import org.apache.hadoop.hbase.HRegionLocation;
70  import org.apache.hadoop.hbase.RemoteExceptionHandler;
71  import org.apache.hadoop.hbase.ServerName;
72  import org.apache.hadoop.hbase.TableName;
73  import org.apache.hadoop.hbase.TableNotFoundException;
74  import org.apache.hadoop.hbase.TableStateManager;
75  import org.apache.hadoop.hbase.classification.InterfaceAudience;
76  import org.apache.hadoop.hbase.client.ConnectionUtils;
77  import org.apache.hadoop.hbase.client.Delete;
78  import org.apache.hadoop.hbase.client.Durability;
79  import org.apache.hadoop.hbase.client.HConnection;
80  import org.apache.hadoop.hbase.client.HConnectionManager;
81  import org.apache.hadoop.hbase.client.Mutation;
82  import org.apache.hadoop.hbase.client.Put;
83  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
84  import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
85  import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
86  import org.apache.hadoop.hbase.io.HeapSize;
87  import org.apache.hadoop.hbase.master.SplitLogManager;
88  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
89  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
90  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
91  import org.apache.hadoop.hbase.protobuf.RequestConverter;
92  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
93  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
94  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
95  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
96  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
97  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
98  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
99  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
100 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
101 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
102 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
103 import org.apache.hadoop.hbase.regionserver.HRegion;
104 import org.apache.hadoop.hbase.regionserver.LastSequenceId;
105 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
106 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
107 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
108 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
109 import org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink;
110 import org.apache.hadoop.hbase.util.Bytes;
111 import org.apache.hadoop.hbase.util.CancelableProgressable;
112 import org.apache.hadoop.hbase.util.ClassSize;
113 import org.apache.hadoop.hbase.util.CommonFSUtils;
114 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
115 import org.apache.hadoop.hbase.util.FSUtils;
116 import org.apache.hadoop.hbase.util.Pair;
117 import org.apache.hadoop.hbase.util.Threads;
118 import org.apache.hadoop.hbase.wal.WAL.Entry;
119 import org.apache.hadoop.hbase.wal.WAL.Reader;
120 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
121 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
122 import org.apache.hadoop.io.MultipleIOException;
123 
124 import com.google.common.base.Preconditions;
125 import com.google.common.collect.Lists;
126 import com.google.protobuf.ServiceException;
127 import com.google.protobuf.TextFormat;
128 
129 /**
130  * This class is responsible for splitting up a bunch of regionserver commit log
131  * files that are no longer being written to, into new files, one per region for
132  * region to replay on startup. Delete the old log files when finished.
133  */
134 @InterfaceAudience.Private
135 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
136   justification="Synchronization on concurrent map is intended")
137 public class WALSplitter {
138   private static final Log LOG = LogFactory.getLog(WALSplitter.class);
139 
140   /** By default we retry errors in splitting, rather than skipping. */
141   public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
142 
143   // Parameters for split process
144   protected final Path walDir;
145   protected final Path rootDir;
146   protected final FileSystem walFS;
147   protected final FileSystem rootFS;
148   protected final Configuration conf;
149 
150   // Major subcomponents of the split process.
151   // These are separated into inner classes to make testing easier.
152   PipelineController controller;
153   OutputSink outputSink;
154   EntryBuffers entryBuffers;
155 
156   private Set<TableName> disablingOrDisabledTables =
157       new HashSet<TableName>();
158   private BaseCoordinatedStateManager csm;
159   private final WALFactory walFactory;
160 
161   private MonitoredTask status;
162 
163   // For checking the latest flushed sequence id
164   protected final LastSequenceId sequenceIdChecker;
165 
166   protected boolean distributedLogReplay;
167 
168   private final boolean splitWriterCreationBounded;
169 
170 
171   // Map encodedRegionName -> lastFlushedSequenceId
172   protected Map<String, Long> lastFlushedSequenceIds = new ConcurrentHashMap<String, Long>();
173 
174   // Map encodedRegionName -> maxSeqIdInStores
175   protected Map<String, Map<byte[], Long>> regionMaxSeqIdInStores =
176       new ConcurrentHashMap<String, Map<byte[], Long>>();
177 
178   // Failed region server that the wal file being split belongs to
179   protected String failedServerName = "";
180 
181   // Number of writer threads
182   private final int numWriterThreads;
183 
184   // Min batch size when replay WAL edits
185   private final int minBatchSize;
186 
187   // the file being split currently
188   private FileStatus fileBeingSplit;
189 
190   public final static String SPLIT_WRITER_CREATION_BOUNDED = "hbase.split.writer.creation.bounded";
191 
192   WALSplitter(final WALFactory factory, Configuration conf, Path walDir, FileSystem walFS,
193       Path rootDir, FileSystem rootFS, LastSequenceId idChecker, CoordinatedStateManager csm,
194       RecoveryMode mode) {
195     this.conf = HBaseConfiguration.create(conf);
196     String codecClassName = conf
197         .get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
198     this.conf.set(HConstants.RPC_CODEC_CONF_KEY, codecClassName);
199     this.walDir = walDir;
200     this.walFS = walFS;
201     this.rootDir = rootDir;
202     this.rootFS = rootFS;
203     this.sequenceIdChecker = idChecker;
204     this.csm = (BaseCoordinatedStateManager)csm;
205     this.walFactory = factory;
206     this.controller = new PipelineController();
207     this.splitWriterCreationBounded = conf.getBoolean(SPLIT_WRITER_CREATION_BOUNDED, false);
208     entryBuffers = new EntryBuffers(controller,
209         this.conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
210             128*1024*1024), splitWriterCreationBounded);
211 
212     // a larger minBatchSize may slow down recovery because replay writer has to wait for
213     // enough edits before replaying them
214     this.minBatchSize = this.conf.getInt("hbase.regionserver.wal.logreplay.batch.size", 64);
215     this.distributedLogReplay = (RecoveryMode.LOG_REPLAY == mode);
216 
217     this.numWriterThreads = this.conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
218     if (csm != null && this.distributedLogReplay) {
219       outputSink = new LogReplayOutputSink(controller, entryBuffers, numWriterThreads);
220     } else {
221       if (this.distributedLogReplay) {
222         LOG.info("ZooKeeperWatcher is passed in as NULL so disable distrubitedLogRepaly.");
223       }
224       this.distributedLogReplay = false;
225       if(splitWriterCreationBounded){
226         outputSink = new BoundedLogWriterCreationOutputSink(controller,
227             entryBuffers, numWriterThreads);
228       }else {
229         outputSink = new LogRecoveredEditsOutputSink(controller, entryBuffers, numWriterThreads);
230       }
231     }
232 
233   }
234 
235   /**
236    * Splits a WAL file into region's recovered-edits directory.
237    * This is the main entry point for distributed log splitting from SplitLogWorker.
238    * <p>
239    * If the log file has N regions then N recovered.edits files will be produced.
240    * <p>
241    * @param walDir
242    * @param logfile
243    * @param walFS FileSystem to use for WAL reading and splitting
244    * @param conf
245    * @param reporter
246    * @param idChecker
247    * @param cp coordination state manager
248    * @return false if it is interrupted by the progress-able.
249    * @throws IOException
250    */
251   public static boolean splitLogFile(Path walDir, FileStatus logfile, FileSystem walFS,
252       Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
253       CoordinatedStateManager cp, RecoveryMode mode, final WALFactory factory) throws IOException {
254     Path rootDir = CommonFSUtils.getRootDir(conf);
255     FileSystem rootFS = rootDir.getFileSystem(conf);
256     WALSplitter s = new WALSplitter(factory, conf, walDir, walFS, rootDir, rootFS, idChecker, cp,
257         mode);
258     return s.splitLogFile(logfile, reporter);
259   }
260 
261   // A wrapper to split one log folder using the method used by distributed
262   // log splitting. Used by tools and unit tests. It should be package private.
263   // It is public only because UpgradeTo96 and TestWALObserver are in different packages,
264   // which uses this method to do log splitting.
265   public static List<Path> split(Path walRootDir, Path logDir, Path oldLogDir,
266       FileSystem walFs, Configuration conf, final WALFactory factory) throws IOException {
267     final FileStatus[] logfiles = SplitLogManager.getFileList(conf,
268         Collections.singletonList(logDir), null);
269     List<Path> splits = new ArrayList<Path>();
270     if (logfiles != null && logfiles.length > 0) {
271       Path rootDir = CommonFSUtils.getRootDir(conf);
272       FileSystem rootFS = rootDir.getFileSystem(conf);
273       for (FileStatus logfile: logfiles) {
274         WALSplitter s = new WALSplitter(factory, conf, walRootDir, walFs, rootDir, rootFS, null,
275             null, RecoveryMode.LOG_SPLITTING);
276         if (s.splitLogFile(logfile, null)) {
277           finishSplitLogFile(walRootDir, oldLogDir, logfile.getPath(), conf);
278           if (s.outputSink.splits != null) {
279             splits.addAll(s.outputSink.splits);
280           }
281         }
282       }
283     }
284     if (!walFs.delete(logDir, true)) {
285       throw new IOException("Unable to delete src dir: " + logDir);
286     }
287     return splits;
288   }
289 
290   /**
291    * log splitting implementation, splits one log file.
292    * @param logfile should be an actual log file.
293    */
294   boolean splitLogFile(FileStatus logfile, CancelableProgressable reporter) throws IOException {
295     Preconditions.checkState(status == null);
296     Preconditions.checkArgument(logfile.isFile(),
297         "passed in file status is for something other than a regular file.");
298     boolean isCorrupted = false;
299     boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
300       SPLIT_SKIP_ERRORS_DEFAULT);
301     int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
302     Path logPath = logfile.getPath();
303     boolean outputSinkStarted = false;
304     boolean progress_failed = false;
305     int editsCount = 0;
306     int editsSkipped = 0;
307 
308     status =
309         TaskMonitor.get().createStatus(
310           "Splitting log file " + logfile.getPath() + "into a temporary staging area.");
311     status.enableStatusJournal(true);
312     Reader in = null;
313     this.fileBeingSplit = logfile;
314     try {
315       long logLength = logfile.getLen();
316       LOG.info("Splitting wal: " + logPath + ", length=" + logLength);
317       LOG.info("DistributedLogReplay = " + this.distributedLogReplay);
318       status.setStatus("Opening log file " + logPath);
319       if (reporter != null && !reporter.progress()) {
320         progress_failed = true;
321         return false;
322       }
323       try {
324         in = getReader(logfile, skipErrors, reporter);
325       } catch (CorruptedLogFileException e) {
326         LOG.warn("Could not get reader, corrupted log file " + logPath, e);
327         ZKSplitLog.markCorrupted(walDir, logfile.getPath().getName(), walFS);
328         isCorrupted = true;
329       }
330       if (in == null) {
331         LOG.warn("Nothing to split in log file " + logPath);
332         return true;
333       }
334       if (csm != null) {
335         try {
336           TableStateManager tsm = csm.getTableStateManager();
337           disablingOrDisabledTables = tsm.getTablesInStates(
338           ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
339         } catch (CoordinatedStateException e) {
340           throw new IOException("Can't get disabling/disabled tables", e);
341         }
342       }
343       int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
344       int numOpenedFilesLastCheck = 0;
345       outputSink.setReporter(reporter);
346       outputSink.setStatus(status);
347       outputSink.startWriterThreads();
348       outputSinkStarted = true;
349       Entry entry;
350       Long lastFlushedSequenceId = -1L;
351       ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(logPath);
352       failedServerName = (serverName == null) ? "" : serverName.getServerName();
353       while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) {
354         byte[] region = entry.getKey().getEncodedRegionName();
355         String encodedRegionNameAsStr = Bytes.toString(region);
356         lastFlushedSequenceId = lastFlushedSequenceIds.get(encodedRegionNameAsStr);
357         if (lastFlushedSequenceId == null) {
358           if (!(isRegionDirPresentUnderRoot(entry.getKey().getTablename(),
359               encodedRegionNameAsStr))) {
360             // The region directory itself is not present in the FS. This indicates that
361             // region/table is already removed. We can skip all the edits for this region.
362             // Setting lastFlushedSequenceId as Long.MAX_VALUE so that all edits will get
363             // skipped by the seqId check below. See more details in HBASE-24189
364             LOG.info(encodedRegionNameAsStr
365                 + " no longer available in the FS. Skipping all edits for this region.");
366             lastFlushedSequenceId = Long.MAX_VALUE;
367           } else {
368             if (this.distributedLogReplay) {
369               RegionStoreSequenceIds ids = csm.getSplitLogWorkerCoordination()
370                   .getRegionFlushedSequenceId(failedServerName, encodedRegionNameAsStr);
371               if (ids != null) {
372                 lastFlushedSequenceId = ids.getLastFlushedSequenceId();
373                 if (LOG.isDebugEnabled()) {
374                   LOG.debug("DLR Last flushed sequenceid for " + encodedRegionNameAsStr + ": "
375                       + TextFormat.shortDebugString(ids));
376                 }
377               }
378             } else if (sequenceIdChecker != null) {
379               RegionStoreSequenceIds ids = sequenceIdChecker.getLastSequenceId(region);
380               Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
381                   Bytes.BYTES_COMPARATOR);
382               for (StoreSequenceId storeSeqId : ids.getStoreSequenceIdList()) {
383                 maxSeqIdInStores.put(storeSeqId.getFamilyName().toByteArray(),
384                     storeSeqId.getSequenceId());
385               }
386               regionMaxSeqIdInStores.put(encodedRegionNameAsStr, maxSeqIdInStores);
387               lastFlushedSequenceId = ids.getLastFlushedSequenceId();
388               if (LOG.isDebugEnabled()) {
389                 LOG.debug("DLS Last flushed sequenceid for " + encodedRegionNameAsStr + ": "
390                     + TextFormat.shortDebugString(ids));
391               }
392             }
393             if (lastFlushedSequenceId == null) {
394               lastFlushedSequenceId = -1L;
395             }
396           }
397           lastFlushedSequenceIds.put(encodedRegionNameAsStr, lastFlushedSequenceId);
398         }
399         if (lastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
400           editsSkipped++;
401           continue;
402         }
403         // Don't send Compaction/Close/Open region events to recovered edit type sinks.
404         if (entry.getEdit().isMetaEdit() && !outputSink.keepRegionEvent(entry)) {
405           editsSkipped++;
406           continue;
407         }
408         entryBuffers.appendEntry(entry);
409         editsCount++;
410         int moreWritersFromLastCheck = this.getNumOpenWriters() - numOpenedFilesLastCheck;
411         // If sufficient edits have passed, check if we should report progress.
412         if (editsCount % interval == 0
413             || moreWritersFromLastCheck > numOpenedFilesBeforeReporting) {
414           numOpenedFilesLastCheck = this.getNumOpenWriters();
415           String countsStr = (editsCount - (editsSkipped + outputSink.getSkippedEdits()))
416               + " edits, skipped " + editsSkipped + " edits.";
417           status.setStatus("Split " + countsStr);
418           if (reporter != null && !reporter.progress()) {
419             progress_failed = true;
420             return false;
421           }
422         }
423       }
424     } catch (InterruptedException ie) {
425       IOException iie = new InterruptedIOException();
426       iie.initCause(ie);
427       throw iie;
428     } catch (CorruptedLogFileException e) {
429       LOG.warn("Could not parse, corrupted log file " + logPath, e);
430       csm.getSplitLogWorkerCoordination().markCorrupted(walDir,
431         logfile.getPath().getName(), walFS);
432       isCorrupted = true;
433     } catch (IOException e) {
434       e = RemoteExceptionHandler.checkIOException(e);
435       throw e;
436     } finally {
437       final String log = "Finishing writing output logs and closing down";
438       LOG.debug(log);
439       status.setStatus(log);
440       try {
441         if (null != in) {
442           in.close();
443         }
444       } catch (IOException exception) {
445         LOG.warn("Could not close wal reader: " + exception.getMessage());
446         LOG.debug("exception details", exception);
447       }
448       try {
449         if (outputSinkStarted) {
450           // Set progress_failed to true as the immediate following statement will reset its value
451           // when finishWritingAndClose() throws exception, progress_failed has the right value
452           progress_failed = true;
453           progress_failed = outputSink.finishWritingAndClose() == null;
454         }
455       } finally {
456         String msg =
457             "Processed " + editsCount + " edits across " + outputSink.getNumberOfRecoveredRegions()
458                 + " regions; edits skipped=" + editsSkipped + "; log file=" + logPath +
459                 ", length=" + logfile.getLen() + // See if length got updated post lease recovery
460                 ", corrupted=" + isCorrupted + ", progress failed=" + progress_failed;
461         LOG.info(msg);
462         status.markComplete(msg);
463         if (LOG.isDebugEnabled()) {
464           LOG.debug("WAL split completed for " + logPath + " , Journal Log: "
465             + status.prettyPrintJournal());
466         }
467       }
468     }
469     return !progress_failed;
470   }
471 
472   private boolean isRegionDirPresentUnderRoot(TableName tableName, String regionName)
473       throws IOException {
474     Path regionDirPath = CommonFSUtils.getRegionDir(this.rootDir, tableName, regionName);
475     return this.rootFS.exists(regionDirPath);
476   }
477 
478   /**
479    * Completes the work done by splitLogFile by archiving logs
480    * <p>
481    * It is invoked by SplitLogManager once it knows that one of the
482    * SplitLogWorkers have completed the splitLogFile() part. If the master
483    * crashes then this function might get called multiple times.
484    * <p>
485    * @param logfile
486    * @param conf
487    * @throws IOException
488    */
489   public static void finishSplitLogFile(String logfile,
490       Configuration conf)  throws IOException {
491     Path walDir = FSUtils.getWALRootDir(conf);
492     Path oldLogDir = new Path(walDir, HConstants.HREGION_OLDLOGDIR_NAME);
493     Path logPath;
494     if (FSUtils.isStartingWithPath(walDir, logfile)) {
495       logPath = new Path(logfile);
496     } else {
497       logPath = new Path(walDir, logfile);
498     }
499     finishSplitLogFile(walDir, oldLogDir, logPath, conf);
500   }
501 
502   private static void finishSplitLogFile(Path walDir, Path oldLogDir,
503       Path logPath, Configuration conf) throws IOException {
504     List<Path> processedLogs = new ArrayList<Path>();
505     List<Path> corruptedLogs = new ArrayList<Path>();
506     FileSystem walFS = walDir.getFileSystem(conf);
507     if (ZKSplitLog.isCorrupted(walDir, logPath.getName(), walFS)) {
508       corruptedLogs.add(logPath);
509     } else {
510       processedLogs.add(logPath);
511     }
512     archiveLogs(corruptedLogs, processedLogs, oldLogDir, walFS, conf);
513     Path stagingDir = ZKSplitLog.getSplitLogDir(walDir, logPath.getName());
514     walFS.delete(stagingDir, true);
515   }
516 
517   /**
518    * Moves processed logs to a oldLogDir after successful processing Moves
519    * corrupted logs (any log that couldn't be successfully parsed to corruptDir
520    * (.corrupt) for later investigation
521    *
522    * @param corruptedLogs
523    * @param processedLogs
524    * @param oldLogDir
525    * @param walFS FileSystem to use for WAL archival
526    * @param conf
527    * @throws IOException
528    */
529   private static void archiveLogs(
530       final List<Path> corruptedLogs,
531       final List<Path> processedLogs, final Path oldLogDir,
532       final FileSystem walFS, final Configuration conf) throws IOException {
533     final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), conf.get(
534         "hbase.regionserver.hlog.splitlog.corrupt.dir",  HConstants.CORRUPT_DIR_NAME));
535 
536     if (!walFS.mkdirs(corruptDir)) {
537       LOG.info("Unable to mkdir " + corruptDir);
538     }
539     walFS.mkdirs(oldLogDir);
540 
541     // this method can get restarted or called multiple times for archiving
542     // the same log files.
543     for (Path corrupted : corruptedLogs) {
544       Path p = new Path(corruptDir, corrupted.getName());
545       if (walFS.exists(corrupted)) {
546         if (!walFS.rename(corrupted, p)) {
547           LOG.warn("Unable to move corrupted log " + corrupted + " to " + p);
548         } else {
549           LOG.warn("Moved corrupted log " + corrupted + " to " + p);
550         }
551       }
552     }
553 
554     for (Path p : processedLogs) {
555       Path newPath = FSHLog.getWALArchivePath(oldLogDir, p);
556       if (walFS.exists(p)) {
557         if (!FSUtils.renameAndSetModifyTime(walFS, p, newPath)) {
558           LOG.warn("Unable to move  " + p + " to " + newPath);
559         } else {
560           LOG.info("Archived processed log " + p + " to " + newPath);
561         }
562       }
563     }
564   }
565 
566   /**
567    * Path to a file under RECOVERED_EDITS_DIR directory of the region found in
568    * <code>logEntry</code> named for the sequenceid in the passed
569    * <code>logEntry</code>: e.g. /hbase/some_table/2323432434/recovered.edits/2332.
570    * This method also ensures existence of RECOVERED_EDITS_DIR under the region
571    * creating it if necessary.
572    * @param logEntry
573    * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
574    * @param tmpDirName of the directory used to sideline old recovered edits file
575    * @param conf
576    * @return Path to file into which to dump split log edits.
577    * @throws IOException
578    */
579   @SuppressWarnings("deprecation")
580   static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
581       String tmpDirName, Configuration conf) throws IOException {
582     FileSystem walFS = FSUtils.getWALFileSystem(conf);
583     TableName tableName = logEntry.getKey().getTablename();
584     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
585     Path regionDir = FSUtils.getWALRegionDir(conf, tableName, encodedRegionName);
586     Path dir = getRegionDirRecoveredEditsDir(regionDir);
587 
588     if (walFS.exists(dir) && walFS.isFile(dir)) {
589       Path tmp = new Path(tmpDirName);
590       if (!walFS.exists(tmp)) {
591         walFS.mkdirs(tmp);
592       }
593       tmp = new Path(tmp, HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
594       LOG.warn("Found existing old file: " + dir + ". It could be some "
595         + "leftover of an old installation. It should be a folder instead. "
596         + "So moving it to " + tmp);
597       if (!walFS.rename(dir, tmp)) {
598         LOG.warn("Failed to sideline old file " + dir);
599       }
600     }
601 
602     if (!walFS.exists(dir) && !walFS.mkdirs(dir)) {
603       LOG.warn("mkdir failed on " + dir);
604     }
605     // Append fileBeingSplit to prevent name conflict since we may have duplicate wal entries now.
606     // Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
607     // region's replayRecoveredEdits will not delete it
608     String fileName = formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum());
609     fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileNameBeingSplit);
610     return new Path(dir, fileName);
611   }
612 
613   private static String getTmpRecoveredEditsFileName(String fileName) {
614     return fileName + RECOVERED_LOG_TMPFILE_SUFFIX;
615   }
616 
617   /**
618    * Get the completed recovered edits file path, renaming it to be by last edit
619    * in the file from its first edit. Then we could use the name to skip
620    * recovered edits when doing {@link HRegion#replayRecoveredEditsIfAny}.
621    * @param srcPath
622    * @param maximumEditLogSeqNum
623    * @return dstPath take file's last edit log seq num as the name
624    */
625   private static Path getCompletedRecoveredEditsFilePath(Path srcPath,
626       long maximumEditLogSeqNum) {
627     String fileName = formatRecoveredEditsFileName(maximumEditLogSeqNum);
628     return new Path(srcPath.getParent(), fileName);
629   }
630 
631   static String formatRecoveredEditsFileName(final long seqid) {
632     return String.format("%019d", seqid);
633   }
634 
635   private static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
636   private static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
637 
638   /**
639    * @param regionDir
640    *          This regions directory in the filesystem.
641    * @return The directory that holds recovered edits files for the region
642    *         <code>regionDir</code>
643    */
644   public static Path getRegionDirRecoveredEditsDir(final Path regionDir) {
645     return new Path(regionDir, HConstants.RECOVERED_EDITS_DIR);
646   }
647 
648   /**
649    * Returns sorted set of edit files made by splitter, excluding files
650    * with '.temp' suffix.
651    *
652    * @param walFS FileSystem to use for reading Recovered edits files
653    * @param regionDir Directory where Recovered edits should reside
654    * @return Files in passed <code>regionDir</code> as a sorted set.
655    * @throws IOException
656    */
657   public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem walFS,
658       final Path regionDir) throws IOException {
659     NavigableSet<Path> filesSorted = new TreeSet<Path>();
660     Path editsdir = getRegionDirRecoveredEditsDir(regionDir);
661     if (!walFS.exists(editsdir)) {
662       return filesSorted;
663     }
664     FileStatus[] files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
665       @Override
666       public boolean accept(Path p) {
667         boolean result = false;
668         try {
669           // Return files and only files that match the editfile names pattern.
670           // There can be other files in this directory other than edit files.
671           // In particular, on error, we'll move aside the bad edit file giving
672           // it a timestamp suffix. See moveAsideBadEditsFile.
673           Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
674           result = walFS.isFile(p) && m.matches();
675           // Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
676           // because it means splitwal thread is writting this file.
677           if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
678             result = false;
679           }
680           // Skip SeqId Files
681           if (isSequenceIdFile(p)) {
682             result = false;
683           }
684         } catch (IOException e) {
685           LOG.warn("Failed isFile check on " + p);
686         }
687         return result;
688       }
689     });
690     if (files == null) {
691       return filesSorted;
692     }
693     for (FileStatus status : files) {
694       filesSorted.add(status.getPath());
695     }
696     return filesSorted;
697   }
698 
699   /**
700    * Move aside a bad edits file.
701    *
702    * @param walFS FileSystem to use for WAL operations
703    * @param edits
704    *          Edits file to move aside.
705    * @return The name of the moved aside file.
706    * @throws IOException
707    */
708   public static Path moveAsideBadEditsFile(final FileSystem walFS, final Path edits)
709       throws IOException {
710     Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
711         + System.currentTimeMillis());
712     if (!walFS.rename(edits, moveAsideName)) {
713       LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
714     }
715     return moveAsideName;
716   }
717 
718   private static final String SEQUENCE_ID_FILE_SUFFIX = ".seqid";
719   private static final String OLD_SEQUENCE_ID_FILE_SUFFIX = "_seqid";
720   private static final int SEQUENCE_ID_FILE_SUFFIX_LENGTH = SEQUENCE_ID_FILE_SUFFIX.length();
721 
722   /**
723    * Is the given file a region open sequence id file.
724    */
725   public static boolean isSequenceIdFile(final Path file) {
726     return file.getName().endsWith(SEQUENCE_ID_FILE_SUFFIX)
727         || file.getName().endsWith(OLD_SEQUENCE_ID_FILE_SUFFIX);
728   }
729 
730   /**
731    * Create a file with name as region open sequence id
732    * @param walFS FileSystem to write Sequence file to
733    * @param regionDir WALRegionDir used to determine where to write edits files
734    * @param newSeqId
735    * @param saftyBumper
736    * @return long new sequence Id value
737    * @throws IOException
738    */
739   public static long writeRegionSequenceIdFile(final FileSystem walFS, final Path regionDir,
740       long newSeqId, long saftyBumper) throws IOException {
741 
742     Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDir);
743     long maxSeqId = 0;
744     FileStatus[] files = null;
745     if (walFS.exists(editsdir)) {
746       files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
747         @Override
748         public boolean accept(Path p) {
749           return isSequenceIdFile(p);
750         }
751       });
752       if (files != null) {
753         for (FileStatus status : files) {
754           String fileName = status.getPath().getName();
755           try {
756             Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length()
757                 - SEQUENCE_ID_FILE_SUFFIX_LENGTH));
758             maxSeqId = Math.max(tmpSeqId, maxSeqId);
759           } catch (NumberFormatException ex) {
760             LOG.warn("Invalid SeqId File Name=" + fileName);
761           }
762         }
763       }
764     }
765     if (maxSeqId > newSeqId) {
766       newSeqId = maxSeqId;
767     }
768     newSeqId += saftyBumper; // bump up SeqId
769 
770     // write a new seqId file
771     Path newSeqIdFile = new Path(editsdir, newSeqId + SEQUENCE_ID_FILE_SUFFIX);
772     if (newSeqId != maxSeqId) {
773       try {
774         if (!walFS.createNewFile(newSeqIdFile) && !walFS.exists(newSeqIdFile)) {
775           throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
776         }
777         if (LOG.isDebugEnabled()) {
778           LOG.debug("Wrote region seqId=" + newSeqIdFile + " to file, newSeqId=" + newSeqId
779               + ", maxSeqId=" + maxSeqId);
780         }
781       } catch (FileAlreadyExistsException ignored) {
782         // latest hdfs throws this exception. it's all right if newSeqIdFile already exists
783       }
784     }
785     // remove old ones
786     if (files != null) {
787       for (FileStatus status : files) {
788         if (newSeqIdFile.equals(status.getPath())) {
789           continue;
790         }
791         walFS.delete(status.getPath(), false);
792       }
793     }
794     return newSeqId;
795   }
796 
797   /**
798    * Create a new {@link Reader} for reading logs to split.
799    *
800    * @param file
801    * @return A new Reader instance, caller should close
802    * @throws IOException
803    * @throws CorruptedLogFileException
804    */
805   protected Reader getReader(FileStatus file, boolean skipErrors, CancelableProgressable reporter)
806       throws IOException, CorruptedLogFileException {
807     Path path = file.getPath();
808     long length = file.getLen();
809     Reader in;
810 
811     // Check for possibly empty file. With appends, currently Hadoop reports a
812     // zero length even if the file has been sync'd. Revisit if HDFS-376 or
813     // HDFS-878 is committed.
814     if (length <= 0) {
815       LOG.warn("File " + path + " might be still open, length is 0");
816     }
817 
818     try {
819       FSUtils.getInstance(walFS, conf).recoverFileLease(walFS, path, conf, reporter);
820       try {
821         in = getReader(path, reporter);
822       } catch (EOFException e) {
823         if (length <= 0) {
824           // TODO should we ignore an empty, not-last log file if skip.errors
825           // is false? Either way, the caller should decide what to do. E.g.
826           // ignore if this is the last log in sequence.
827           // TODO is this scenario still possible if the log has been
828           // recovered (i.e. closed)
829           LOG.warn("Could not open " + path + " for reading. File is empty", e);
830           return null;
831         } else {
832           // EOFException being ignored
833           return null;
834         }
835       }
836     } catch (IOException e) {
837       if (e instanceof FileNotFoundException) {
838         // A wal file may not exist anymore. Nothing can be recovered so move on
839         LOG.warn("File " + path + " doesn't exist anymore.", e);
840         return null;
841       }
842       if (!skipErrors || e instanceof InterruptedIOException) {
843         throw e; // Don't mark the file corrupted if interrupted, or not skipErrors
844       }
845       throw new CorruptedLogFileException("skipErrors=true Could not open wal "
846         + path + " ignoring", e);
847     }
848     return in;
849   }
850 
851   static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
852   throws CorruptedLogFileException, IOException {
853     try {
854       return in.next();
855     } catch (EOFException eof) {
856       // truncated files are expected if a RS crashes (see HBASE-2643)
857       LOG.info("EOF from wal " + path + ".  continuing");
858       return null;
859     } catch (IOException e) {
860       // If the IOE resulted from bad file format,
861       // then this problem is idempotent and retrying won't help
862       if (e.getCause() != null &&
863           (e.getCause() instanceof ParseException ||
864            e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
865         LOG.warn("Parse exception " + e.getCause().toString() + " from wal "
866            + path + ".  continuing");
867         return null;
868       }
869       if (!skipErrors) {
870         throw e;
871       }
872       throw new CorruptedLogFileException("skipErrors=true Ignoring exception"
873         + " while parsing wal " + path + ". Marking as corrupted", e);
874     }
875   }
876 
877   /**
878    * Create a new {@link Writer} for writing log splits.
879    * @return a new Writer instance, caller should close
880    */
881   protected Writer createWriter(Path logfile)
882       throws IOException {
883     return walFactory.createRecoveredEditsWriter(walFS, logfile);
884   }
885 
886   /**
887    * Create a new {@link Reader} for reading logs to split.
888    * @return new Reader instance, caller should close
889    */
890   protected Reader getReader(Path curLogFile, CancelableProgressable reporter) throws IOException {
891     return walFactory.createReader(walFS, curLogFile, reporter);
892   }
893 
894   /**
895    * Get current open writers
896    */
897   private int getNumOpenWriters() {
898     int result = 0;
899     if (this.outputSink != null) {
900       result += this.outputSink.getNumOpenWriters();
901     }
902     return result;
903   }
904 
905   /**
906    * Contains some methods to control WAL-entries producer / consumer interactions
907    */
908   public static class PipelineController {
909     // If an exception is thrown by one of the other threads, it will be
910     // stored here.
911     AtomicReference<Throwable> thrown = new AtomicReference<Throwable>();
912 
913     // Wait/notify for when data has been produced by the writer thread,
914     // consumed by the reader thread, or an exception occurred
915     public final Object dataAvailable = new Object();
916 
917     void writerThreadError(Throwable t) {
918       thrown.compareAndSet(null, t);
919     }
920 
921     /**
922      * Check for errors in the writer threads. If any is found, rethrow it.
923      */
924     void checkForErrors() throws IOException {
925       Throwable thrown = this.thrown.get();
926       if (thrown == null) return;
927       if (thrown instanceof IOException) {
928         throw new IOException(thrown);
929       } else {
930         throw new RuntimeException(thrown);
931       }
932     }
933   }
934 
935   /**
936    * Class which accumulates edits and separates them into a buffer per region
937    * while simultaneously accounting RAM usage. Blocks if the RAM usage crosses
938    * a predefined threshold.
939    *
940    * Writer threads then pull region-specific buffers from this class.
941    */
942   public static class EntryBuffers {
943     PipelineController controller;
944 
945     Map<byte[], RegionEntryBuffer> buffers =
946       new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
947 
948     /* Track which regions are currently in the middle of writing. We don't allow
949        an IO thread to pick up bytes from a region if we're already writing
950        data for that region in a different IO thread. */
951     Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
952 
953     long totalBuffered = 0;
954     final long maxHeapUsage;
955     boolean splitWriterCreationBounded;
956 
957 
958     public EntryBuffers(PipelineController controller, long maxHeapUsage) {
959       this(controller, maxHeapUsage, false);
960     }
961 
962     public EntryBuffers(PipelineController controller, long maxHeapUsage,
963         boolean splitWriterCreationBounded) {
964       this.controller = controller;
965       this.maxHeapUsage = maxHeapUsage;
966       this.splitWriterCreationBounded = splitWriterCreationBounded;
967     }
968 
969     /**
970      * Append a log entry into the corresponding region buffer.
971      * Blocks if the total heap usage has crossed the specified threshold.
972      *
973      * @throws InterruptedException
974      * @throws IOException
975      */
976     public void appendEntry(Entry entry) throws InterruptedException, IOException {
977       WALKey key = entry.getKey();
978 
979       RegionEntryBuffer buffer;
980       long incrHeap;
981       synchronized (this) {
982         buffer = buffers.get(key.getEncodedRegionName());
983         if (buffer == null) {
984           buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
985           buffers.put(key.getEncodedRegionName(), buffer);
986         }
987         incrHeap= buffer.appendEntry(entry);
988       }
989 
990       // If we crossed the chunk threshold, wait for more space to be available
991       synchronized (controller.dataAvailable) {
992         totalBuffered += incrHeap;
993         while (totalBuffered > maxHeapUsage && controller.thrown.get() == null) {
994           LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
995           controller.dataAvailable.wait(2000);
996         }
997         controller.dataAvailable.notifyAll();
998       }
999       controller.checkForErrors();
1000     }
1001 
1002     /**
1003      * @return RegionEntryBuffer a buffer of edits to be written or replayed.
1004      */
1005     synchronized RegionEntryBuffer getChunkToWrite() {
1006       // The core part of limiting opening writers is it doesn't return chunk only if the heap size
1007       // is over maxHeapUsage. Thus it doesn't need to create a writer for each region
1008       // during splitting. It will flush all the logs in the buffer after splitting through a
1009       // threadpool, which means the number of writers it created is under control
1010       if(splitWriterCreationBounded && totalBuffered < maxHeapUsage){
1011         return null;
1012       }
1013 
1014       long biggestSize = 0;
1015       byte[] biggestBufferKey = null;
1016 
1017       for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
1018         long size = entry.getValue().heapSize();
1019         if (size > biggestSize && (!currentlyWriting.contains(entry.getKey()))) {
1020           biggestSize = size;
1021           biggestBufferKey = entry.getKey();
1022         }
1023       }
1024       if (biggestBufferKey == null) {
1025         return null;
1026       }
1027 
1028       RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
1029       currentlyWriting.add(biggestBufferKey);
1030       return buffer;
1031     }
1032 
1033     void doneWriting(RegionEntryBuffer buffer) {
1034       synchronized (this) {
1035         boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
1036         assert removed;
1037       }
1038       long size = buffer.heapSize();
1039 
1040       synchronized (controller.dataAvailable) {
1041         totalBuffered -= size;
1042         // We may unblock writers
1043         controller.dataAvailable.notifyAll();
1044       }
1045     }
1046 
1047     synchronized boolean isRegionCurrentlyWriting(byte[] region) {
1048       return currentlyWriting.contains(region);
1049     }
1050 
1051     public void waitUntilDrained() {
1052       synchronized (controller.dataAvailable) {
1053         while (totalBuffered > 0) {
1054           try {
1055             controller.dataAvailable.wait(2000);
1056           } catch (InterruptedException e) {
1057             LOG.warn("Got interrupted while waiting for EntryBuffers is drained");
1058             Thread.interrupted();
1059             break;
1060           }
1061         }
1062       }
1063     }
1064   }
1065 
1066   /**
1067    * A buffer of some number of edits for a given region.
1068    * This accumulates edits and also provides a memory optimization in order to
1069    * share a single byte array instance for the table and region name.
1070    * Also tracks memory usage of the accumulated edits.
1071    */
1072   public static class RegionEntryBuffer implements HeapSize {
1073     long heapInBuffer = 0;
1074     List<Entry> entryBuffer;
1075     TableName tableName;
1076     byte[] encodedRegionName;
1077 
1078     RegionEntryBuffer(TableName tableName, byte[] region) {
1079       this.tableName = tableName;
1080       this.encodedRegionName = region;
1081       this.entryBuffer = new LinkedList<Entry>();
1082     }
1083 
1084     long appendEntry(Entry entry) {
1085       internify(entry);
1086       entryBuffer.add(entry);
1087       long incrHeap = entry.getEdit().heapSize() +
1088         ClassSize.align(2 * ClassSize.REFERENCE) + // WALKey pointers
1089         0; // TODO linkedlist entry
1090       heapInBuffer += incrHeap;
1091       return incrHeap;
1092     }
1093 
1094     private void internify(Entry entry) {
1095       WALKey k = entry.getKey();
1096       k.internTableName(this.tableName);
1097       k.internEncodedRegionName(this.encodedRegionName);
1098     }
1099 
1100     @Override
1101     public long heapSize() {
1102       return heapInBuffer;
1103     }
1104 
1105     public byte[] getEncodedRegionName() {
1106       return encodedRegionName;
1107     }
1108 
1109     public List<Entry> getEntryBuffer() {
1110       return entryBuffer;
1111     }
1112 
1113     public TableName getTableName() {
1114       return tableName;
1115     }
1116   }
1117 
1118   public static class WriterThread extends Thread {
1119     private volatile boolean shouldStop = false;
1120     private PipelineController controller;
1121     private EntryBuffers entryBuffers;
1122     private OutputSink outputSink = null;
1123 
1124     WriterThread(PipelineController controller, EntryBuffers entryBuffers, OutputSink sink, int i){
1125       super(Thread.currentThread().getName() + "-Writer-" + i);
1126       this.controller = controller;
1127       this.entryBuffers = entryBuffers;
1128       outputSink = sink;
1129     }
1130 
1131     @Override
1132     public void run()  {
1133       try {
1134         doRun();
1135       } catch (Throwable t) {
1136         LOG.error("Exiting thread", t);
1137         controller.writerThreadError(t);
1138       }
1139     }
1140 
1141     private void doRun() throws IOException {
1142       if (LOG.isTraceEnabled()) LOG.trace("Writer thread starting");
1143       while (true) {
1144         RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
1145         if (buffer == null) {
1146           // No data currently available, wait on some more to show up
1147           synchronized (controller.dataAvailable) {
1148             if (shouldStop && !this.outputSink.flush()) {
1149               return;
1150             }
1151             try {
1152               controller.dataAvailable.wait(500);
1153             } catch (InterruptedException ie) {
1154               if (!shouldStop) {
1155                 throw new RuntimeException(ie);
1156               }
1157             }
1158           }
1159           continue;
1160         }
1161 
1162         assert buffer != null;
1163         try {
1164           writeBuffer(buffer);
1165         } finally {
1166           entryBuffers.doneWriting(buffer);
1167         }
1168       }
1169     }
1170 
1171     private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
1172       outputSink.append(buffer);
1173     }
1174 
1175     void finish() {
1176       synchronized (controller.dataAvailable) {
1177         shouldStop = true;
1178         controller.dataAvailable.notifyAll();
1179       }
1180     }
1181   }
1182 
1183   /**
1184    * The following class is an abstraction class to provide a common interface to support both
1185    * existing recovered edits file sink and region server WAL edits replay sink
1186    */
1187   public static abstract class OutputSink {
1188 
1189     protected PipelineController controller;
1190     protected EntryBuffers entryBuffers;
1191 
1192     protected ConcurrentHashMap<String, SinkWriter> writers = new ConcurrentHashMap<>();
1193     protected ConcurrentHashMap<String, Long> regionMaximumEditLogSeqNum =
1194         new ConcurrentHashMap<>();
1195 
1196     protected final List<WriterThread> writerThreads = Lists.newArrayList();
1197 
1198     /* Set of regions which we've decided should not output edits */
1199     protected final Set<byte[]> blacklistedRegions = Collections
1200         .synchronizedSet(new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR));
1201 
1202     protected boolean closeAndCleanCompleted = false;
1203 
1204     protected boolean writersClosed = false;
1205 
1206     protected final int numThreads;
1207 
1208     protected CancelableProgressable reporter = null;
1209 
1210     protected AtomicLong skippedEdits = new AtomicLong();
1211 
1212     protected List<Path> splits = null;
1213 
1214     protected MonitoredTask status = null;
1215 
1216     public OutputSink(PipelineController controller, EntryBuffers entryBuffers, int numWriters) {
1217       numThreads = numWriters;
1218       this.controller = controller;
1219       this.entryBuffers = entryBuffers;
1220     }
1221 
1222     void setReporter(CancelableProgressable reporter) {
1223       this.reporter = reporter;
1224     }
1225 
1226     /**
1227      * Start the threads that will pump data from the entryBuffers to the output files.
1228      */
1229     public synchronized void startWriterThreads() {
1230       for (int i = 0; i < numThreads; i++) {
1231         WriterThread t = new WriterThread(controller, entryBuffers, this, i);
1232         t.start();
1233         writerThreads.add(t);
1234       }
1235     }
1236 
1237     /**
1238      *
1239      * Update region's maximum edit log SeqNum.
1240      */
1241     void updateRegionMaximumEditLogSeqNum(Entry entry) {
1242 
1243       synchronized (regionMaximumEditLogSeqNum) {
1244         String encodedRegionName = Bytes.toString(entry.getKey().getEncodedRegionName());
1245         Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(encodedRegionName);
1246         if (currentMaxSeqNum == null || entry.getKey().getLogSeqNum() > currentMaxSeqNum) {
1247           regionMaximumEditLogSeqNum.put(encodedRegionName, entry.getKey()
1248               .getLogSeqNum());
1249         }
1250       }
1251     }
1252 
1253     Long getRegionMaximumEditLogSeqNum(byte[] region) {
1254       return regionMaximumEditLogSeqNum.get(Bytes.toString(region));
1255     }
1256 
1257     /**
1258      * @return the number of currently opened writers
1259      */
1260     int getNumOpenWriters() {
1261       return this.writers.size();
1262     }
1263 
1264     long getSkippedEdits() {
1265       return this.skippedEdits.get();
1266     }
1267 
1268     void setStatus(MonitoredTask status) {
1269       this.status = status;
1270     }
1271 
1272     /**
1273      * Wait for writer threads to dump all info to the sink
1274      * @return true when there is no error
1275      * @throws IOException
1276      */
1277     protected boolean finishWriting(boolean interrupt) throws IOException {
1278       LOG.debug("Waiting for split writer threads to finish");
1279       boolean progress_failed = false;
1280       for (WriterThread t : writerThreads) {
1281         t.finish();
1282       }
1283       if (interrupt) {
1284         for (WriterThread t : writerThreads) {
1285           t.interrupt(); // interrupt the writer threads. We are stopping now.
1286         }
1287       }
1288 
1289       for (WriterThread t : writerThreads) {
1290         if (!progress_failed && reporter != null && !reporter.progress()) {
1291           progress_failed = true;
1292         }
1293         try {
1294           t.join();
1295         } catch (InterruptedException ie) {
1296           IOException iie = new InterruptedIOException();
1297           iie.initCause(ie);
1298           throw iie;
1299         }
1300       }
1301       controller.checkForErrors();
1302       final String msg = this.writerThreads.size() + " split writer threads finished";
1303       LOG.info(msg);
1304       updateStatusWithMsg(msg);
1305       return (!progress_failed);
1306     }
1307 
1308     public abstract List<Path> finishWritingAndClose() throws IOException;
1309 
1310     /**
1311      * @return a map from encoded region ID to the number of edits written out for that region.
1312      */
1313     public abstract Map<byte[], Long> getOutputCounts();
1314 
1315     /**
1316      * @return number of regions we've recovered
1317      */
1318     public abstract int getNumberOfRecoveredRegions();
1319 
1320     /**
1321      * @param buffer A WAL Edit Entry
1322      * @throws IOException
1323      */
1324     public abstract void append(RegionEntryBuffer buffer) throws IOException;
1325 
1326     /**
1327      * WriterThread call this function to help flush internal remaining edits in buffer before close
1328      * @return true when underlying sink has something to flush
1329      */
1330     public boolean flush() throws IOException {
1331       return false;
1332     }
1333 
1334     /**
1335      * Some WALEdit's contain only KV's for account on what happened to a region.
1336      * Not all sinks will want to get all of those edits.
1337      *
1338      * @return Return true if this sink wants to accept this region-level WALEdit.
1339      */
1340     public abstract boolean keepRegionEvent(Entry entry);
1341 
1342     /**
1343      * Set status message in {@link MonitoredTask} instance that is set in this OutputSink
1344      *
1345      * @param msg message to update the status with
1346      */
1347     protected final void updateStatusWithMsg(String msg) {
1348       if (status != null) {
1349         status.setStatus(msg);
1350       }
1351     }
1352   }
1353 
1354   /**
1355    * Class that manages the output streams from the log splitting process.
1356    */
1357   class LogRecoveredEditsOutputSink extends OutputSink {
1358 
1359     public LogRecoveredEditsOutputSink(PipelineController controller, EntryBuffers entryBuffers,
1360         int numWriters) {
1361       // More threads could potentially write faster at the expense
1362       // of causing more disk seeks as the logs are split.
1363       // 3. After a certain setting (probably around 3) the
1364       // process will be bound on the reader in the current
1365       // implementation anyway.
1366       super(controller, entryBuffers, numWriters);
1367     }
1368 
1369     /**
1370      * @return null if failed to report progress
1371      * @throws IOException
1372      */
1373     @Override
1374     public List<Path> finishWritingAndClose() throws IOException {
1375       boolean isSuccessful = false;
1376       List<Path> result = null;
1377       try {
1378         isSuccessful = finishWriting(false);
1379       } finally {
1380         result = close();
1381         List<IOException> thrown = closeLogWriters(null);
1382         if (thrown != null && !thrown.isEmpty()) {
1383           throw MultipleIOException.createIOException(thrown);
1384         }
1385       }
1386       if (isSuccessful) {
1387         splits = result;
1388       }
1389       return splits;
1390     }
1391 
1392     // delete the one with fewer wal entries
1393     void deleteOneWithFewerEntries(WriterAndPath wap, Path dst)
1394         throws IOException {
1395       long dstMinLogSeqNum = -1L;
1396       try (WAL.Reader reader = walFactory.createReader(walFS, dst)) {
1397         WAL.Entry entry = reader.next();
1398         if (entry != null) {
1399           dstMinLogSeqNum = entry.getKey().getLogSeqNum();
1400         }
1401       } catch (EOFException e) {
1402         if (LOG.isDebugEnabled()) {
1403           LOG.debug(
1404             "Got EOF when reading first WAL entry from " + dst + ", an empty or broken WAL file?",
1405             e);
1406         }
1407       }
1408       if (wap.minLogSeqNum < dstMinLogSeqNum) {
1409         final String errorMsg =
1410           "Found existing old edits file. It could be the result of a previous failed"
1411             + " split attempt or we have duplicated wal entries. Deleting " + dst + ", length="
1412             + walFS.getFileStatus(dst).getLen();
1413         LOG.warn(errorMsg);
1414         updateStatusWithMsg(errorMsg);
1415         if (!walFS.delete(dst, false)) {
1416           final String msg = "Failed deleting of old " + dst;
1417           LOG.warn(msg);
1418           updateStatusWithMsg(msg);
1419           throw new IOException("Failed deleting of old " + dst);
1420         }
1421       } else {
1422         final String errorMsg =
1423           "Found existing old edits file and we have less entries. Deleting " + wap.p + ", length="
1424             + walFS.getFileStatus(wap.p).getLen();
1425         LOG.warn(errorMsg);
1426         updateStatusWithMsg(errorMsg);
1427         if (!walFS.delete(wap.p, false)) {
1428           final String failureMsg = "Failed deleting of " + wap.p;
1429           LOG.warn(failureMsg);
1430           updateStatusWithMsg(failureMsg);
1431           throw new IOException(failureMsg);
1432         }
1433       }
1434     }
1435 
1436     /**
1437      * Close all of the output streams.
1438      * @return the list of paths written.
1439      */
1440     List<Path> close() throws IOException {
1441       Preconditions.checkState(!closeAndCleanCompleted);
1442 
1443       final List<Path> paths = new ArrayList<Path>();
1444       final List<IOException> thrown = Lists.newArrayList();
1445       ThreadPoolExecutor closeThreadPool = Threads.getBoundedCachedThreadPool(numThreads, 30L,
1446         TimeUnit.SECONDS, new ThreadFactory() {
1447           private int count = 1;
1448 
1449           @Override
1450           public Thread newThread(Runnable r) {
1451             Thread t = new Thread(r, "split-log-closeStream-" + count++);
1452             return t;
1453           }
1454         });
1455       CompletionService<Void> completionService =
1456         new ExecutorCompletionService<Void>(closeThreadPool);
1457       boolean progress_failed;
1458       try{
1459         progress_failed = executeCloseTask(completionService, thrown, paths);
1460       } catch (InterruptedException e) {
1461         IOException iie = new InterruptedIOException();
1462         iie.initCause(e);
1463         throw iie;
1464       } catch (ExecutionException e) {
1465         throw new IOException(e.getCause());
1466       } finally {
1467         closeThreadPool.shutdownNow();
1468       }
1469 
1470       if (!thrown.isEmpty()) {
1471         throw MultipleIOException.createIOException(thrown);
1472       }
1473       writersClosed = true;
1474       closeAndCleanCompleted = true;
1475       if (progress_failed) {
1476         return null;
1477       }
1478       return paths;
1479     }
1480 
1481     boolean executeCloseTask(CompletionService<Void> completionService,
1482         final List<IOException> thrown, final List<Path> paths)
1483         throws InterruptedException, ExecutionException {
1484       for (final Map.Entry<String, SinkWriter> writersEntry : writers.entrySet()) {
1485         if (LOG.isTraceEnabled()) {
1486           LOG.trace("Submitting close of " + ((WriterAndPath) writersEntry.getValue()).p);
1487         }
1488         completionService.submit(new Callable<Void>() {
1489           @Override public Void call() throws Exception {
1490             WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
1491             Path dst = closeWriter(writersEntry.getKey(), wap, thrown);
1492             paths.add(dst);
1493             return null;
1494           }
1495         });
1496       }
1497       boolean progress_failed = false;
1498       for (int i = 0, n = this.writers.size(); i < n; i++) {
1499         Future<Void> future = completionService.take();
1500         future.get();
1501         if (!progress_failed && reporter != null && !reporter.progress()) {
1502           progress_failed = true;
1503         }
1504       }
1505       return progress_failed;
1506     }
1507 
1508     Path closeWriter(String encodedRegionName, WriterAndPath wap, List<IOException> thrown)
1509         throws IOException {
1510       if (LOG.isTraceEnabled()) {
1511         LOG.trace("Closing " + wap.p);
1512       }
1513 
1514       try {
1515         wap.w.close();
1516       } catch (IOException ioe) {
1517         final String errorMsg = "Couldn't close log at " + wap.p;
1518         LOG.error(errorMsg, ioe);
1519         updateStatusWithMsg(errorMsg);
1520         thrown.add(ioe);
1521         return null;
1522       }
1523       final String msg =
1524         "Closed wap " + wap.p + " (wrote " + wap.editsWritten + " edits, skipped "
1525           + wap.editsSkipped + " edits in " + (wap.nanosSpent / 1000 / 1000) + "ms";
1526       if (LOG.isDebugEnabled()) {
1527         LOG.debug(msg);
1528       }
1529       updateStatusWithMsg(msg);
1530       if (wap.editsWritten == 0) {
1531         // just remove the empty recovered.edits file
1532         if (walFS.exists(wap.p) && !walFS.delete(wap.p, false)) {
1533           final String errorMsg = "Failed deleting empty " + wap.p;
1534           LOG.warn(errorMsg);
1535           throw new IOException("Failed deleting empty  " + wap.p);
1536         }
1537         return null;
1538       }
1539 
1540       Path dst = getCompletedRecoveredEditsFilePath(wap.p,
1541           regionMaximumEditLogSeqNum.get(encodedRegionName));
1542       try {
1543         if (!dst.equals(wap.p) && walFS.exists(dst)) {
1544           deleteOneWithFewerEntries(wap, dst);
1545         }
1546         // Skip the unit tests which create a splitter that reads and
1547         // writes the data without touching disk.
1548         // TestHLogSplit#testThreading is an example.
1549         if (walFS.exists(wap.p)) {
1550           if (!walFS.rename(wap.p, dst)) {
1551             final String errorMsg = "Failed renaming " + wap.p + " to " + dst;
1552             updateStatusWithMsg(errorMsg);
1553             throw new IOException(errorMsg);
1554           }
1555           final String renameLog = "Rename " + wap.p + " to " + dst;
1556           LOG.info(renameLog);
1557           updateStatusWithMsg(renameLog);
1558         }
1559       } catch (IOException ioe) {
1560         final String errorMsg = "Couldn't rename " + wap.p + " to " + dst;
1561         LOG.error(errorMsg, ioe);
1562         updateStatusWithMsg(errorMsg);
1563         thrown.add(ioe);
1564         return null;
1565       }
1566       return dst;
1567     }
1568 
1569 
1570     private List<IOException> closeLogWriters(List<IOException> thrown) throws IOException {
1571       if (writersClosed) {
1572         return thrown;
1573       }
1574 
1575       if (thrown == null) {
1576         thrown = Lists.newArrayList();
1577       }
1578       try {
1579         for (WriterThread t : writerThreads) {
1580           while (t.isAlive()) {
1581             t.shouldStop = true;
1582             t.interrupt();
1583             try {
1584               t.join(10);
1585             } catch (InterruptedException e) {
1586               IOException iie = new InterruptedIOException();
1587               iie.initCause(e);
1588               throw iie;
1589             }
1590           }
1591         }
1592       } finally {
1593         WriterAndPath wap = null;
1594         for (SinkWriter tmpWAP : writers.values()) {
1595           try {
1596             wap = (WriterAndPath) tmpWAP;
1597             wap.w.close();
1598           } catch (IOException ioe) {
1599             final String errorMsg = "Couldn't close log at " + wap.p;
1600             LOG.error(errorMsg, ioe);
1601             updateStatusWithMsg(errorMsg);
1602             thrown.add(ioe);
1603             continue;
1604           }
1605           final String msg =
1606             "Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in " + (wap.nanosSpent
1607               / 1000 / 1000) + "ms)";
1608           LOG.info(msg);
1609           updateStatusWithMsg(msg);
1610         }
1611         writersClosed = true;
1612       }
1613 
1614       return thrown;
1615     }
1616 
1617     /**
1618      * Get a writer and path for a log starting at the given entry. This function is threadsafe so
1619      * long as multiple threads are always acting on different regions.
1620      * @return null if this region shouldn't output any logs
1621      */
1622     WriterAndPath getWriterAndPath(Entry entry, boolean reusable) throws IOException {
1623       byte region[] = entry.getKey().getEncodedRegionName();
1624       String regionName = Bytes.toString(region);
1625       WriterAndPath ret = (WriterAndPath) writers.get(regionName);
1626       if (ret != null) {
1627         return ret;
1628       }
1629       // If we already decided that this region doesn't get any output
1630       // we don't need to check again.
1631       if (blacklistedRegions.contains(region)) {
1632         return null;
1633       }
1634       ret = createWAP(region, entry);
1635       if (ret == null) {
1636         blacklistedRegions.add(region);
1637         return null;
1638       }
1639 
1640       if(reusable) {
1641         writers.put(regionName, ret);
1642       }
1643       return ret;
1644     }
1645 
1646     /**
1647      * @return a path with a write for that path. caller should close.
1648      */
1649     WriterAndPath createWAP(byte[] region, Entry entry) throws IOException {
1650       String tmpDirName = conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
1651         HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
1652       Path regionedits = getRegionSplitEditsPath(entry,
1653           fileBeingSplit.getPath().getName(), tmpDirName, conf);
1654       if (regionedits == null) {
1655         return null;
1656       }
1657       if (walFS.exists(regionedits)) {
1658         final String warnMsg = "Found old edits file. It could be the "
1659           + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
1660           + walFS.getFileStatus(regionedits).getLen();
1661         LOG.warn(warnMsg);
1662         updateStatusWithMsg(warnMsg);
1663         if (!walFS.delete(regionedits, false)) {
1664           final String errorMsg = "Failed delete of old " + regionedits;
1665           LOG.warn(errorMsg);
1666           updateStatusWithMsg(errorMsg);
1667         }
1668       }
1669       Writer w = createWriter(regionedits);
1670       final String msg = "Creating writer path=" + regionedits;
1671       LOG.debug(msg);
1672       updateStatusWithMsg(msg);
1673       return new WriterAndPath(regionedits, w, entry.getKey().getLogSeqNum());
1674     }
1675 
1676     void filterCellByStore(Entry logEntry) {
1677       Map<byte[], Long> maxSeqIdInStores =
1678           regionMaxSeqIdInStores.get(Bytes.toString(logEntry.getKey().getEncodedRegionName()));
1679       if (maxSeqIdInStores == null || maxSeqIdInStores.isEmpty()) {
1680         return;
1681       }
1682       // Create the array list for the cells that aren't filtered.
1683       // We make the assumption that most cells will be kept.
1684       ArrayList<Cell> keptCells = new ArrayList<Cell>(logEntry.getEdit().getCells().size());
1685       for (Cell cell : logEntry.getEdit().getCells()) {
1686         if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
1687           keptCells.add(cell);
1688         } else {
1689           byte[] family = CellUtil.cloneFamily(cell);
1690           Long maxSeqId = maxSeqIdInStores.get(family);
1691           // Do not skip cell even if maxSeqId is null. Maybe we are in a rolling upgrade,
1692           // or the master was crashed before and we can not get the information.
1693           if (maxSeqId == null || maxSeqId.longValue() < logEntry.getKey().getLogSeqNum()) {
1694             keptCells.add(cell);
1695           }
1696         }
1697       }
1698 
1699       // Anything in the keptCells array list is still live.
1700       // So rather than removing the cells from the array list
1701       // which would be an O(n^2) operation, we just replace the list
1702       logEntry.getEdit().setCells(keptCells);
1703     }
1704 
1705     @Override
1706     public void append(RegionEntryBuffer buffer) throws IOException {
1707       appendBuffer(buffer, true);
1708     }
1709 
1710 
1711     WriterAndPath appendBuffer(RegionEntryBuffer buffer, boolean reusable) throws IOException{
1712       List<Entry> entries = buffer.entryBuffer;
1713       if (entries.isEmpty()) {
1714         LOG.warn("got an empty buffer, skipping");
1715         return null;
1716       }
1717       WriterAndPath wap = null;
1718 
1719       long startTime = System.nanoTime();
1720       int editsCount = 0;
1721 
1722       for (Entry logEntry : entries) {
1723         try {
1724           if (wap == null) {
1725             wap = getWriterAndPath(logEntry, reusable);
1726             if (wap == null) {
1727               if (LOG.isDebugEnabled()) {
1728                 LOG.debug("getWriterAndPath decided we don't need to write edits for " + logEntry);
1729               }
1730               return null;
1731             }
1732           }
1733           filterCellByStore(logEntry);
1734           if (!logEntry.getEdit().isEmpty()) {
1735             wap.w.append(logEntry);
1736             this.updateRegionMaximumEditLogSeqNum(logEntry);
1737             editsCount++;
1738           } else {
1739             wap.incrementSkippedEdits(1);
1740           }
1741         } catch (IOException e) {
1742           logAndThrowWriterAppendFailure(logEntry, e);
1743         }
1744       }
1745       // Pass along summary statistics
1746       wap.incrementEdits(editsCount);
1747       wap.incrementNanoTime(System.nanoTime() - startTime);
1748       return wap;
1749     }
1750 
1751     private void logAndThrowWriterAppendFailure(Entry logEntry, IOException e) throws IOException {
1752       e = RemoteExceptionHandler.checkIOException(e);
1753       final String errorMsg = "Failed to write log entry " + logEntry.toString() + " to log";
1754       LOG.fatal(errorMsg, e);
1755       updateStatusWithMsg(errorMsg);
1756       throw e;
1757     }
1758 
1759     @Override
1760     public boolean keepRegionEvent(Entry entry) {
1761       ArrayList<Cell> cells = entry.getEdit().getCells();
1762       for (int i = 0; i < cells.size(); i++) {
1763         if (WALEdit.isCompactionMarker(cells.get(i))) {
1764           return true;
1765         }
1766       }
1767       return false;
1768     }
1769 
1770     /**
1771      * @return a map from encoded region ID to the number of edits written out for that region.
1772      */
1773     @Override
1774     public Map<byte[], Long> getOutputCounts() {
1775       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1776       synchronized (writers) {
1777         for (Map.Entry<String, SinkWriter> entry : writers.entrySet()) {
1778           ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
1779         }
1780       }
1781       return ret;
1782     }
1783 
1784     @Override
1785     public int getNumberOfRecoveredRegions() {
1786       return writers.size();
1787     }
1788   }
1789 
1790   /**
1791    * Class that will limit the number of hdfs writers we create to split the logs
1792    */
1793   class BoundedLogWriterCreationOutputSink extends LogRecoveredEditsOutputSink{
1794 
1795     ConcurrentHashMap<String, Long> regionRecoverStatMap = new ConcurrentHashMap<>();
1796 
1797 
1798     public BoundedLogWriterCreationOutputSink(PipelineController controller,
1799         EntryBuffers entryBuffers, int numWriters){
1800       super(controller, entryBuffers, numWriters);
1801     }
1802 
1803     @Override
1804     public List<Path> finishWritingAndClose() throws IOException {
1805       boolean isSuccessful;
1806       List<Path> result;
1807       try {
1808         isSuccessful = finishWriting(false);
1809       } finally {
1810         result = close();
1811       }
1812       if (isSuccessful) {
1813         splits = result;
1814       }
1815       return splits;
1816     }
1817 
1818     @Override
1819     boolean executeCloseTask(CompletionService<Void> closeCompletionService,
1820         final List<IOException> thrown, final List<Path> paths)
1821         throws InterruptedException, ExecutionException {
1822       for (final Map.Entry<byte[], RegionEntryBuffer> buffer : entryBuffers.buffers.entrySet()) {
1823         LOG.info("Submitting write then close of " +
1824             Bytes.toString(buffer.getValue().encodedRegionName));
1825         closeCompletionService.submit(new Callable<Void>() {
1826           public Void call() throws Exception {
1827             Path dst = writeThenClose(buffer.getValue());
1828             paths.add(dst);
1829             return null;
1830           }
1831         });
1832       }
1833 
1834       boolean progress_failed = false;
1835       for (int i = 0, n = entryBuffers.buffers.size(); i < n; i++) {
1836         Future<Void> future = closeCompletionService.take();
1837         future.get();
1838         if (!progress_failed && reporter != null && !reporter.progress()) {
1839           progress_failed = true;
1840         }
1841       }
1842       return progress_failed;
1843     }
1844 
1845     @Override
1846     public Map<byte[], Long> getOutputCounts() {
1847       Map<byte[], Long> regionRecoverStatMapResult = new HashMap<>();
1848       for(Map.Entry<String, Long> entry: regionRecoverStatMap.entrySet()){
1849         regionRecoverStatMapResult.put(Bytes.toBytes(entry.getKey()), entry.getValue());
1850       }
1851       return regionRecoverStatMapResult;
1852     }
1853 
1854     @Override
1855     public int getNumberOfRecoveredRegions() {
1856       return regionRecoverStatMap.size();
1857     }
1858 
1859     @Override
1860     public void append(RegionEntryBuffer buffer) throws IOException {
1861       writeThenClose(buffer);
1862     }
1863 
1864     private Path writeThenClose(RegionEntryBuffer buffer) throws IOException {
1865       WriterAndPath wap = appendBuffer(buffer, false);
1866       Path dst = null;
1867       if(wap != null){
1868         String encodedRegionName = Bytes.toString(buffer.encodedRegionName);
1869         Long value = regionRecoverStatMap.putIfAbsent(encodedRegionName, wap.editsWritten);
1870         if(value != null){
1871           Long newValue = regionRecoverStatMap.get(encodedRegionName) + wap.editsWritten;
1872           regionRecoverStatMap.put(encodedRegionName, newValue);
1873         }
1874       }
1875 
1876       List<IOException> thrown = new ArrayList<>();
1877       if(wap != null) {
1878         dst = closeWriter(Bytes.toString(buffer.encodedRegionName), wap, thrown);
1879       }
1880 
1881       if(!thrown.isEmpty()){
1882         throw MultipleIOException.createIOException(thrown);
1883       }
1884       return dst;
1885     }
1886   }
1887 
1888 
1889   /**
1890    * Class wraps the actual writer which writes data out and related statistics
1891    */
1892   public abstract static class SinkWriter {
1893     /* Count of edits written to this path */
1894     long editsWritten = 0;
1895     /* Count of edits skipped to this path */
1896     long editsSkipped = 0;
1897     /* Number of nanos spent writing to this log */
1898     long nanosSpent = 0;
1899 
1900     void incrementEdits(int edits) {
1901       editsWritten += edits;
1902     }
1903 
1904     void incrementSkippedEdits(int skipped) {
1905       editsSkipped += skipped;
1906     }
1907 
1908     void incrementNanoTime(long nanos) {
1909       nanosSpent += nanos;
1910     }
1911   }
1912 
1913   /**
1914    * Private data structure that wraps a Writer and its Path, also collecting statistics about the
1915    * data written to this output.
1916    */
1917   private final static class WriterAndPath extends SinkWriter {
1918     final Path p;
1919     final Writer w;
1920     final long minLogSeqNum;
1921 
1922     WriterAndPath(final Path p, final Writer w, final long minLogSeqNum) {
1923       this.p = p;
1924       this.w = w;
1925       this.minLogSeqNum = minLogSeqNum;
1926     }
1927   }
1928 
1929   /**
1930    * Class that manages to replay edits from WAL files directly to assigned fail over region servers
1931    */
1932   class LogReplayOutputSink extends OutputSink {
1933     private static final double BUFFER_THRESHOLD = 0.35;
1934     private static final String KEY_DELIMITER = "#";
1935 
1936     private long waitRegionOnlineTimeOut;
1937     private final Set<String> recoveredRegions = Collections.synchronizedSet(new HashSet<String>());
1938     private final Map<String, RegionServerWriter> rsWriters =
1939         new ConcurrentHashMap<String, RegionServerWriter>();
1940     // online encoded region name -> region location map
1941     private final Map<String, HRegionLocation> onlineRegions =
1942         new ConcurrentHashMap<String, HRegionLocation>();
1943 
1944     private final Map<TableName, HConnection> tableNameToHConnectionMap = Collections
1945         .synchronizedMap(new TreeMap<TableName, HConnection>());
1946     /**
1947      * Map key -> value layout
1948      * servername:table name -> Queue(Row)
1949      */
1950     private final Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
1951         new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();
1952     private final List<Throwable> thrown = new ArrayList<Throwable>();
1953 
1954     // The following sink is used in distrubitedLogReplay mode for entries of regions in a disabling
1955     // table. It's a limitation of distributedLogReplay. Because log replay needs a region is
1956     // assigned and online before it can replay wal edits while regions of disabling/disabled table
1957     // won't be assigned by AM. We can retire this code after HBASE-8234.
1958     private LogRecoveredEditsOutputSink logRecoveredEditsOutputSink;
1959     private boolean hasEditsInDisablingOrDisabledTables = false;
1960 
1961     public LogReplayOutputSink(PipelineController controller, EntryBuffers entryBuffers,
1962         int numWriters) {
1963       super(controller, entryBuffers, numWriters);
1964       this.waitRegionOnlineTimeOut = conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT,
1965           ZKSplitLogManagerCoordination.DEFAULT_TIMEOUT);
1966       this.logRecoveredEditsOutputSink = new LogRecoveredEditsOutputSink(controller,
1967         entryBuffers, numWriters);
1968       this.logRecoveredEditsOutputSink.setReporter(reporter);
1969     }
1970 
1971     @Override
1972     public void append(RegionEntryBuffer buffer) throws IOException {
1973       List<Entry> entries = buffer.entryBuffer;
1974       if (entries.isEmpty()) {
1975         LOG.warn("got an empty buffer, skipping");
1976         return;
1977       }
1978 
1979       // check if current region in a disabling or disabled table
1980       if (disablingOrDisabledTables.contains(buffer.tableName)) {
1981         // need fall back to old way
1982         logRecoveredEditsOutputSink.append(buffer);
1983         hasEditsInDisablingOrDisabledTables = true;
1984         // store regions we have recovered so far
1985         addToRecoveredRegions(Bytes.toString(buffer.encodedRegionName));
1986         return;
1987       }
1988 
1989       // group entries by region servers
1990       groupEditsByServer(entries);
1991 
1992       // process workitems
1993       String maxLocKey = null;
1994       int maxSize = 0;
1995       List<Pair<HRegionLocation, Entry>> maxQueue = null;
1996       synchronized (this.serverToBufferQueueMap) {
1997         for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry:
1998               serverToBufferQueueMap.entrySet()) {
1999           List<Pair<HRegionLocation, Entry>> curQueue = entry.getValue();
2000           if (curQueue.size() > maxSize) {
2001             maxSize = curQueue.size();
2002             maxQueue = curQueue;
2003             maxLocKey = entry.getKey();
2004           }
2005         }
2006         if (maxSize < minBatchSize
2007             && entryBuffers.totalBuffered < BUFFER_THRESHOLD * entryBuffers.maxHeapUsage) {
2008           // buffer more to process
2009           return;
2010         } else if (maxSize > 0) {
2011           this.serverToBufferQueueMap.remove(maxLocKey);
2012         }
2013       }
2014 
2015       if (maxSize > 0) {
2016         processWorkItems(maxLocKey, maxQueue);
2017       }
2018     }
2019 
2020     private void addToRecoveredRegions(String encodedRegionName) {
2021       if (!recoveredRegions.contains(encodedRegionName)) {
2022         recoveredRegions.add(encodedRegionName);
2023       }
2024     }
2025 
2026     /**
2027      * Helper function to group WALEntries to individual region servers
2028      * @throws IOException
2029      */
2030     private void groupEditsByServer(List<Entry> entries) throws IOException {
2031       Set<TableName> nonExistentTables = null;
2032       Long cachedLastFlushedSequenceId = -1l;
2033       for (Entry entry : entries) {
2034         WALEdit edit = entry.getEdit();
2035         TableName table = entry.getKey().getTablename();
2036         // clear scopes which isn't needed for recovery
2037         entry.getKey().setScopes(null);
2038         String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
2039         // skip edits of non-existent tables
2040         if (nonExistentTables != null && nonExistentTables.contains(table)) {
2041           this.skippedEdits.incrementAndGet();
2042           continue;
2043         }
2044 
2045         Map<byte[], Long> maxStoreSequenceIds = null;
2046         boolean needSkip = false;
2047         HRegionLocation loc = null;
2048         String locKey = null;
2049         List<Cell> cells = edit.getCells();
2050         List<Cell> skippedCells = new ArrayList<Cell>();
2051         HConnection hconn = this.getConnectionByTableName(table);
2052 
2053         for (Cell cell : cells) {
2054           byte[] row = CellUtil.cloneRow(cell);
2055           byte[] family = CellUtil.cloneFamily(cell);
2056           boolean isCompactionEntry = false;
2057           if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
2058             CompactionDescriptor compaction = WALEdit.getCompaction(cell);
2059             if (compaction != null && compaction.hasRegionName()) {
2060               try {
2061                 byte[][] regionName = HRegionInfo.parseRegionName(compaction.getRegionName()
2062                   .toByteArray());
2063                 row = regionName[1]; // startKey of the region
2064                 family = compaction.getFamilyName().toByteArray();
2065                 isCompactionEntry = true;
2066               } catch (Exception ex) {
2067                 LOG.warn("Unexpected exception received, ignoring " + ex);
2068                 skippedCells.add(cell);
2069                 continue;
2070               }
2071             } else {
2072               skippedCells.add(cell);
2073               continue;
2074             }
2075           }
2076 
2077           try {
2078             loc =
2079                 locateRegionAndRefreshLastFlushedSequenceId(hconn, table, row,
2080                   encodeRegionNameStr);
2081             // skip replaying the compaction if the region is gone
2082             if (isCompactionEntry && !encodeRegionNameStr.equalsIgnoreCase(
2083               loc.getRegionInfo().getEncodedName())) {
2084               LOG.info("Not replaying a compaction marker for an older region: "
2085                   + encodeRegionNameStr);
2086               needSkip = true;
2087             }
2088           } catch (TableNotFoundException ex) {
2089             // table has been deleted so skip edits of the table
2090             LOG.info("Table " + table + " doesn't exist. Skip log replay for region "
2091                 + encodeRegionNameStr);
2092             lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE);
2093             if (nonExistentTables == null) {
2094               nonExistentTables = new TreeSet<TableName>();
2095             }
2096             nonExistentTables.add(table);
2097             this.skippedEdits.incrementAndGet();
2098             needSkip = true;
2099             break;
2100           }
2101 
2102           cachedLastFlushedSequenceId =
2103               lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
2104           if (cachedLastFlushedSequenceId != null
2105               && cachedLastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
2106             // skip the whole WAL entry
2107             this.skippedEdits.incrementAndGet();
2108             needSkip = true;
2109             break;
2110           } else {
2111             if (maxStoreSequenceIds == null) {
2112               maxStoreSequenceIds =
2113                   regionMaxSeqIdInStores.get(loc.getRegionInfo().getEncodedName());
2114             }
2115             if (maxStoreSequenceIds != null) {
2116               Long maxStoreSeqId = maxStoreSequenceIds.get(family);
2117               if (maxStoreSeqId == null || maxStoreSeqId >= entry.getKey().getLogSeqNum()) {
2118                 // skip current kv if column family doesn't exist anymore or already flushed
2119                 skippedCells.add(cell);
2120                 continue;
2121               }
2122             }
2123           }
2124         }
2125 
2126         // skip the edit
2127         if (loc == null || needSkip) continue;
2128 
2129         if (!skippedCells.isEmpty()) {
2130           cells.removeAll(skippedCells);
2131         }
2132 
2133         synchronized (serverToBufferQueueMap) {
2134           locKey = loc.getHostnamePort() + KEY_DELIMITER + table;
2135           List<Pair<HRegionLocation, Entry>> queue = serverToBufferQueueMap.get(locKey);
2136           if (queue == null) {
2137             queue =
2138                 Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, Entry>>());
2139             serverToBufferQueueMap.put(locKey, queue);
2140           }
2141           queue.add(new Pair<HRegionLocation, Entry>(loc, entry));
2142         }
2143         // store regions we have recovered so far
2144         addToRecoveredRegions(loc.getRegionInfo().getEncodedName());
2145       }
2146     }
2147 
2148     /**
2149      * Locate destination region based on table name & row. This function also makes sure the
2150      * destination region is online for replay.
2151      * @throws IOException
2152      */
2153     private HRegionLocation locateRegionAndRefreshLastFlushedSequenceId(HConnection hconn,
2154         TableName table, byte[] row, String originalEncodedRegionName) throws IOException {
2155       // fetch location from cache
2156       HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
2157       if(loc != null) return loc;
2158       // fetch location from hbase:meta directly without using cache to avoid hit old dead server
2159       loc = hconn.getRegionLocation(table, row, true);
2160       if (loc == null) {
2161         throw new IOException("Can't locate location for row:" + Bytes.toString(row)
2162             + " of table:" + table);
2163       }
2164       // check if current row moves to a different region due to region merge/split
2165       if (!originalEncodedRegionName.equalsIgnoreCase(loc.getRegionInfo().getEncodedName())) {
2166         // originalEncodedRegionName should have already flushed
2167         lastFlushedSequenceIds.put(originalEncodedRegionName, Long.MAX_VALUE);
2168         HRegionLocation tmpLoc = onlineRegions.get(loc.getRegionInfo().getEncodedName());
2169         if (tmpLoc != null) return tmpLoc;
2170       }
2171 
2172       Long lastFlushedSequenceId = -1l;
2173       AtomicBoolean isRecovering = new AtomicBoolean(true);
2174       loc = waitUntilRegionOnline(loc, row, this.waitRegionOnlineTimeOut, isRecovering);
2175       if (!isRecovering.get()) {
2176         // region isn't in recovering at all because WAL file may contain a region that has
2177         // been moved to somewhere before hosting RS fails
2178         lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), Long.MAX_VALUE);
2179         LOG.info("logReplay skip region: " + loc.getRegionInfo().getEncodedName()
2180             + " because it's not in recovering.");
2181       } else {
2182         Long cachedLastFlushedSequenceId =
2183             lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
2184 
2185         // retrieve last flushed sequence Id from ZK. Because region postOpenDeployTasks will
2186         // update the value for the region
2187         RegionStoreSequenceIds ids =
2188             csm.getSplitLogWorkerCoordination().getRegionFlushedSequenceId(failedServerName,
2189               loc.getRegionInfo().getEncodedName());
2190         if (ids != null) {
2191           lastFlushedSequenceId = ids.getLastFlushedSequenceId();
2192           Map<byte[], Long> storeIds = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2193           List<StoreSequenceId> maxSeqIdInStores = ids.getStoreSequenceIdList();
2194           for (StoreSequenceId id : maxSeqIdInStores) {
2195             storeIds.put(id.getFamilyName().toByteArray(), id.getSequenceId());
2196           }
2197           regionMaxSeqIdInStores.put(loc.getRegionInfo().getEncodedName(), storeIds);
2198         }
2199 
2200         if (cachedLastFlushedSequenceId == null
2201             || lastFlushedSequenceId > cachedLastFlushedSequenceId) {
2202           lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), lastFlushedSequenceId);
2203         }
2204       }
2205 
2206       onlineRegions.put(loc.getRegionInfo().getEncodedName(), loc);
2207       return loc;
2208     }
2209 
2210     private void processWorkItems(String key, List<Pair<HRegionLocation, Entry>> actions)
2211         throws IOException {
2212       RegionServerWriter rsw = null;
2213 
2214       long startTime = System.nanoTime();
2215       try {
2216         rsw = getRegionServerWriter(key);
2217         rsw.sink.replayEntries(actions);
2218 
2219         // Pass along summary statistics
2220         rsw.incrementEdits(actions.size());
2221         rsw.incrementNanoTime(System.nanoTime() - startTime);
2222       } catch (IOException e) {
2223         e = RemoteExceptionHandler.checkIOException(e);
2224         LOG.fatal(" Got while writing log entry to log", e);
2225         throw e;
2226       }
2227     }
2228 
2229     /**
2230      * Wait until region is online on the destination region server
2231      * @param loc
2232      * @param row
2233      * @param timeout How long to wait
2234      * @param isRecovering Recovering state of the region interested on destination region server.
2235      * @return True when region is online on the destination region server
2236      * @throws InterruptedException
2237      */
2238     private HRegionLocation waitUntilRegionOnline(HRegionLocation loc, byte[] row,
2239         final long timeout, AtomicBoolean isRecovering)
2240         throws IOException {
2241       final long endTime = EnvironmentEdgeManager.currentTime() + timeout;
2242       final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
2243         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
2244       boolean reloadLocation = false;
2245       TableName tableName = loc.getRegionInfo().getTable();
2246       int tries = 0;
2247       Throwable cause = null;
2248       while (endTime > EnvironmentEdgeManager.currentTime()) {
2249         try {
2250           // Try and get regioninfo from the hosting server.
2251           HConnection hconn = getConnectionByTableName(tableName);
2252           if(reloadLocation) {
2253             loc = hconn.getRegionLocation(tableName, row, true);
2254           }
2255           BlockingInterface remoteSvr = hconn.getAdmin(loc.getServerName());
2256           HRegionInfo region = loc.getRegionInfo();
2257           try {
2258             GetRegionInfoRequest request =
2259                 RequestConverter.buildGetRegionInfoRequest(region.getRegionName());
2260             GetRegionInfoResponse response = remoteSvr.getRegionInfo(null, request);
2261             if (HRegionInfo.convert(response.getRegionInfo()) != null) {
2262               isRecovering.set((response.hasIsRecovering()) ? response.getIsRecovering() : true);
2263               return loc;
2264             }
2265           } catch (ServiceException se) {
2266             throw ProtobufUtil.getRemoteException(se);
2267           }
2268         } catch (IOException e) {
2269           cause = e.getCause();
2270           if(!(cause instanceof RegionOpeningException)) {
2271             reloadLocation = true;
2272           }
2273         }
2274         long expectedSleep = ConnectionUtils.getPauseTime(pause, tries);
2275         try {
2276           Thread.sleep(expectedSleep);
2277         } catch (InterruptedException e) {
2278           throw new IOException("Interrupted when waiting region " +
2279               loc.getRegionInfo().getEncodedName() + " online.", e);
2280         }
2281         tries++;
2282       }
2283 
2284       throw new IOException("Timeout when waiting region " + loc.getRegionInfo().getEncodedName() +
2285         " online for " + timeout + " milliseconds.", cause);
2286     }
2287 
2288     @Override
2289     public boolean flush() throws IOException {
2290       String curLoc = null;
2291       int curSize = 0;
2292       List<Pair<HRegionLocation, Entry>> curQueue = null;
2293       synchronized (this.serverToBufferQueueMap) {
2294         for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry :
2295                 serverToBufferQueueMap.entrySet()) {
2296           String locationKey = entry.getKey();
2297           curQueue = entry.getValue();
2298           if (!curQueue.isEmpty()) {
2299             curSize = curQueue.size();
2300             curLoc = locationKey;
2301             break;
2302           }
2303         }
2304         if (curSize > 0) {
2305           this.serverToBufferQueueMap.remove(curLoc);
2306         }
2307       }
2308 
2309       if (curSize > 0) {
2310         this.processWorkItems(curLoc, curQueue);
2311         // We should already have control of the monitor; ensure this is the case.
2312         synchronized(controller.dataAvailable) {
2313           controller.dataAvailable.notifyAll();
2314         }
2315         return true;
2316       }
2317       return false;
2318     }
2319 
2320     @Override
2321     public boolean keepRegionEvent(Entry entry) {
2322       return true;
2323     }
2324 
2325     void addWriterError(Throwable t) {
2326       thrown.add(t);
2327     }
2328 
2329     @Override
2330     public List<Path> finishWritingAndClose() throws IOException {
2331       try {
2332         if (!finishWriting(false)) {
2333           return null;
2334         }
2335         if (hasEditsInDisablingOrDisabledTables) {
2336           splits = logRecoveredEditsOutputSink.finishWritingAndClose();
2337         } else {
2338           splits = new ArrayList<Path>();
2339         }
2340         // returns an empty array in order to keep interface same as old way
2341         return splits;
2342       } finally {
2343         List<IOException> thrown = closeRegionServerWriters();
2344         if (thrown != null && !thrown.isEmpty()) {
2345           throw MultipleIOException.createIOException(thrown);
2346         }
2347       }
2348     }
2349 
2350     @Override
2351     int getNumOpenWriters() {
2352       return this.rsWriters.size() + this.logRecoveredEditsOutputSink.getNumOpenWriters();
2353     }
2354 
2355     private List<IOException> closeRegionServerWriters() throws IOException {
2356       List<IOException> result = null;
2357       if (!writersClosed) {
2358         result = Lists.newArrayList();
2359         try {
2360           for (WriterThread t : writerThreads) {
2361             while (t.isAlive()) {
2362               t.shouldStop = true;
2363               t.interrupt();
2364               try {
2365                 t.join(10);
2366               } catch (InterruptedException e) {
2367                 IOException iie = new InterruptedIOException();
2368                 iie.initCause(e);
2369                 throw iie;
2370               }
2371             }
2372           }
2373         } finally {
2374           synchronized (rsWriters) {
2375             for (Map.Entry<String, RegionServerWriter> entry : rsWriters.entrySet()) {
2376               String locationKey = entry.getKey();
2377               RegionServerWriter tmpW = entry.getValue();
2378               try {
2379                 tmpW.close();
2380               } catch (IOException ioe) {
2381                 LOG.error("Couldn't close writer for region server:" + locationKey, ioe);
2382                 result.add(ioe);
2383               }
2384             }
2385           }
2386 
2387           // close connections
2388           synchronized (this.tableNameToHConnectionMap) {
2389             for (Map.Entry<TableName, HConnection> entry :
2390                   tableNameToHConnectionMap.entrySet()) {
2391               TableName tableName = entry.getKey();
2392               HConnection hconn = entry.getValue();
2393               try {
2394                 hconn.clearRegionCache();
2395                 hconn.close();
2396               } catch (IOException ioe) {
2397                 result.add(ioe);
2398               }
2399             }
2400           }
2401           writersClosed = true;
2402         }
2403       }
2404       return result;
2405     }
2406 
2407     @Override
2408     public Map<byte[], Long> getOutputCounts() {
2409       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2410       synchronized (rsWriters) {
2411         for (Map.Entry<String, RegionServerWriter> entry : rsWriters.entrySet()) {
2412           ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
2413         }
2414       }
2415       return ret;
2416     }
2417 
2418     @Override
2419     public int getNumberOfRecoveredRegions() {
2420       return this.recoveredRegions.size();
2421     }
2422 
2423     /**
2424      * Get a writer and path for a log starting at the given entry. This function is threadsafe so
2425      * long as multiple threads are always acting on different regions.
2426      * @return null if this region shouldn't output any logs
2427      */
2428     private RegionServerWriter getRegionServerWriter(String loc) throws IOException {
2429       RegionServerWriter ret = rsWriters.get(loc);
2430       if (ret != null) {
2431         return ret;
2432       }
2433 
2434       TableName tableName = getTableFromLocationStr(loc);
2435       if(tableName == null){
2436         throw new IOException("Invalid location string:" + loc + " found. Replay aborted.");
2437       }
2438 
2439       HConnection hconn = getConnectionByTableName(tableName);
2440       synchronized (rsWriters) {
2441         ret = rsWriters.get(loc);
2442         if (ret == null) {
2443           ret = new RegionServerWriter(conf, tableName, hconn);
2444           rsWriters.put(loc, ret);
2445         }
2446       }
2447       return ret;
2448     }
2449 
2450     private HConnection getConnectionByTableName(final TableName tableName) throws IOException {
2451       HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
2452       if (hconn == null) {
2453         synchronized (this.tableNameToHConnectionMap) {
2454           hconn = this.tableNameToHConnectionMap.get(tableName);
2455           if (hconn == null) {
2456             hconn = HConnectionManager.getConnection(conf);
2457             this.tableNameToHConnectionMap.put(tableName, hconn);
2458           }
2459         }
2460       }
2461       return hconn;
2462     }
2463     private TableName getTableFromLocationStr(String loc) {
2464       /**
2465        * location key is in format <server name:port>#<table name>
2466        */
2467       String[] splits = loc.split(KEY_DELIMITER);
2468       if (splits.length != 2) {
2469         return null;
2470       }
2471       return TableName.valueOf(splits[1]);
2472     }
2473   }
2474 
2475   /**
2476    * Private data structure that wraps a receiving RS and collecting statistics about the data
2477    * written to this newly assigned RS.
2478    */
2479   private final static class RegionServerWriter extends SinkWriter {
2480     final WALEditsReplaySink sink;
2481 
2482     RegionServerWriter(final Configuration conf, final TableName tableName, final HConnection conn)
2483         throws IOException {
2484       this.sink = new WALEditsReplaySink(conf, tableName, conn);
2485     }
2486 
2487     void close() throws IOException {
2488     }
2489   }
2490 
2491   static class CorruptedLogFileException extends Exception {
2492     private static final long serialVersionUID = 1L;
2493 
2494     CorruptedLogFileException(String s) {
2495       super(s);
2496     }
2497 
2498     /**
2499      * CorruptedLogFileException with cause
2500      *
2501      * @param message the message for this exception
2502      * @param cause the cause for this exception
2503      */
2504     CorruptedLogFileException(String message, Throwable cause) {
2505       super(message, cause);
2506     }
2507   }
2508 
2509   /** A struct used by getMutationsFromWALEntry */
2510   public static class MutationReplay implements Comparable<MutationReplay> {
2511     public MutationReplay(MutationType type, Mutation mutation, long nonceGroup, long nonce) {
2512       this.type = type;
2513       this.mutation = mutation;
2514       if(this.mutation.getDurability() != Durability.SKIP_WAL) {
2515         // using ASYNC_WAL for relay
2516         this.mutation.setDurability(Durability.ASYNC_WAL);
2517       }
2518       this.nonceGroup = nonceGroup;
2519       this.nonce = nonce;
2520     }
2521 
2522     public final MutationType type;
2523     public final Mutation mutation;
2524     public final long nonceGroup;
2525     public final long nonce;
2526 
2527     @Override
2528     public int compareTo(final MutationReplay d) {
2529       return this.mutation.compareTo(d.mutation);
2530     }
2531 
2532     @Override
2533     public boolean equals(Object obj) {
2534       if(!(obj instanceof MutationReplay)) {
2535         return false;
2536       } else {
2537         return this.compareTo((MutationReplay)obj) == 0;
2538       }
2539     }
2540 
2541     @Override
2542     public int hashCode() {
2543       return this.mutation.hashCode();
2544     }
2545   }
2546 
2547   /**
2548    * This function is used to construct mutations from a WALEntry. It also
2549    * reconstructs WALKey &amp; WALEdit from the passed in WALEntry
2550    * @param entry
2551    * @param cells
2552    * @param logEntry pair of WALKey and WALEdit instance stores WALKey and WALEdit instances
2553    *          extracted from the passed in WALEntry.
2554    * @param durability
2555    * @return list of Pair&lt;MutationType, Mutation&gt; to be replayed
2556    * @throws IOException
2557    */
2558   public static List<MutationReplay> getMutationsFromWALEntry(WALEntry entry, CellScanner cells,
2559       Pair<WALKey, WALEdit> logEntry, Durability durability)
2560           throws IOException {
2561     if (entry == null) {
2562       // return an empty array
2563       return new ArrayList<MutationReplay>();
2564     }
2565 
2566     long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
2567       entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
2568     int count = entry.getAssociatedCellCount();
2569     List<MutationReplay> mutations = new ArrayList<MutationReplay>();
2570     Cell previousCell = null;
2571     Mutation m = null;
2572     WALKey key = null;
2573     WALEdit val = null;
2574     if (logEntry != null) val = new WALEdit();
2575 
2576     for (int i = 0; i < count; i++) {
2577       // Throw index out of bounds if our cell count is off
2578       if (!cells.advance()) {
2579         throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
2580       }
2581       Cell cell = cells.current();
2582       if (val != null) val.add(cell);
2583 
2584       boolean isNewRowOrType =
2585           previousCell == null || previousCell.getTypeByte() != cell.getTypeByte()
2586               || !CellUtil.matchingRow(previousCell, cell);
2587       if (isNewRowOrType) {
2588         // Create new mutation
2589         if (CellUtil.isDelete(cell)) {
2590           m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2591           // Deletes don't have nonces.
2592           mutations.add(new MutationReplay(
2593               MutationType.DELETE, m, HConstants.NO_NONCE, HConstants.NO_NONCE));
2594         } else {
2595           m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2596           // Puts might come from increment or append, thus we need nonces.
2597           long nonceGroup = entry.getKey().hasNonceGroup()
2598               ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
2599           long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
2600           mutations.add(new MutationReplay(MutationType.PUT, m, nonceGroup, nonce));
2601         }
2602       }
2603       if (CellUtil.isDelete(cell)) {
2604         ((Delete) m).addDeleteMarker(cell);
2605       } else {
2606         ((Put) m).add(cell);
2607       }
2608       m.setDurability(durability);
2609       previousCell = cell;
2610     }
2611 
2612     // reconstruct WALKey
2613     if (logEntry != null) {
2614       org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKeyProto = entry.getKey();
2615       List<UUID> clusterIds = new ArrayList<UUID>(walKeyProto.getClusterIdsCount());
2616       for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
2617         clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
2618       }
2619       // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
2620       key = new HLogKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
2621               walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(),
2622               clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce(), null);
2623       logEntry.setFirst(key);
2624       logEntry.setSecond(val);
2625     }
2626 
2627     return mutations;
2628   }
2629 }