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    * And also set storage policy for RECOVERED_EDITS_DIR if WAL_STORAGE_POLICY is configured.
573    * @param logEntry
574    * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
575    * @param tmpDirName of the directory used to sideline old recovered edits file
576    * @param conf
577    * @return Path to file into which to dump split log edits.
578    * @throws IOException
579    */
580   @SuppressWarnings("deprecation")
581   static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
582       String tmpDirName, Configuration conf) throws IOException {
583     FileSystem walFS = FSUtils.getWALFileSystem(conf);
584     TableName tableName = logEntry.getKey().getTablename();
585     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
586     Path regionDir = FSUtils.getWALRegionDir(conf, tableName, encodedRegionName);
587     Path dir = getRegionDirRecoveredEditsDir(regionDir);
588 
589     if (walFS.exists(dir) && walFS.isFile(dir)) {
590       Path tmp = new Path(tmpDirName);
591       if (!walFS.exists(tmp)) {
592         walFS.mkdirs(tmp);
593       }
594       tmp = new Path(tmp, HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
595       LOG.warn("Found existing old file: " + dir + ". It could be some "
596         + "leftover of an old installation. It should be a folder instead. "
597         + "So moving it to " + tmp);
598       if (!walFS.rename(dir, tmp)) {
599         LOG.warn("Failed to sideline old file " + dir);
600       }
601     }
602 
603     if (!walFS.exists(dir) && !walFS.mkdirs(dir)) {
604       LOG.warn("mkdir failed on " + dir);
605     } else {
606       String storagePolicy =
607         conf.get(HConstants.WAL_STORAGE_POLICY, HConstants.DEFAULT_WAL_STORAGE_POLICY);
608       FSUtils.setStoragePolicy(walFS, dir, storagePolicy);
609     }
610     // Append fileBeingSplit to prevent name conflict since we may have duplicate wal entries now.
611     // Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
612     // region's replayRecoveredEdits will not delete it
613     String fileName = formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum());
614     fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileNameBeingSplit);
615     return new Path(dir, fileName);
616   }
617 
618   private static String getTmpRecoveredEditsFileName(String fileName) {
619     return fileName + RECOVERED_LOG_TMPFILE_SUFFIX;
620   }
621 
622   /**
623    * Get the completed recovered edits file path, renaming it to be by last edit
624    * in the file from its first edit. Then we could use the name to skip
625    * recovered edits when doing {@link HRegion#replayRecoveredEditsIfAny}.
626    * @param srcPath
627    * @param maximumEditLogSeqNum
628    * @return dstPath take file's last edit log seq num as the name
629    */
630   private static Path getCompletedRecoveredEditsFilePath(Path srcPath,
631       long maximumEditLogSeqNum) {
632     String fileName = formatRecoveredEditsFileName(maximumEditLogSeqNum);
633     return new Path(srcPath.getParent(), fileName);
634   }
635 
636   static String formatRecoveredEditsFileName(final long seqid) {
637     return String.format("%019d", seqid);
638   }
639 
640   private static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
641   private static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
642 
643   /**
644    * @param regionDir
645    *          This regions directory in the filesystem.
646    * @return The directory that holds recovered edits files for the region
647    *         <code>regionDir</code>
648    */
649   public static Path getRegionDirRecoveredEditsDir(final Path regionDir) {
650     return new Path(regionDir, HConstants.RECOVERED_EDITS_DIR);
651   }
652 
653   /**
654    * Returns sorted set of edit files made by splitter, excluding files
655    * with '.temp' suffix.
656    *
657    * @param walFS FileSystem to use for reading Recovered edits files
658    * @param regionDir Directory where Recovered edits should reside
659    * @return Files in passed <code>regionDir</code> as a sorted set.
660    * @throws IOException
661    */
662   public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem walFS,
663       final Path regionDir) throws IOException {
664     NavigableSet<Path> filesSorted = new TreeSet<Path>();
665     Path editsdir = getRegionDirRecoveredEditsDir(regionDir);
666     if (!walFS.exists(editsdir)) {
667       return filesSorted;
668     }
669     FileStatus[] files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
670       @Override
671       public boolean accept(Path p) {
672         boolean result = false;
673         try {
674           // Return files and only files that match the editfile names pattern.
675           // There can be other files in this directory other than edit files.
676           // In particular, on error, we'll move aside the bad edit file giving
677           // it a timestamp suffix. See moveAsideBadEditsFile.
678           Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
679           result = walFS.isFile(p) && m.matches();
680           // Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
681           // because it means splitwal thread is writting this file.
682           if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
683             result = false;
684           }
685           // Skip SeqId Files
686           if (isSequenceIdFile(p)) {
687             result = false;
688           }
689         } catch (IOException e) {
690           LOG.warn("Failed isFile check on " + p);
691         }
692         return result;
693       }
694     });
695     if (files == null) {
696       return filesSorted;
697     }
698     for (FileStatus status : files) {
699       filesSorted.add(status.getPath());
700     }
701     return filesSorted;
702   }
703 
704   /**
705    * Move aside a bad edits file.
706    *
707    * @param walFS FileSystem to use for WAL operations
708    * @param edits
709    *          Edits file to move aside.
710    * @return The name of the moved aside file.
711    * @throws IOException
712    */
713   public static Path moveAsideBadEditsFile(final FileSystem walFS, final Path edits)
714       throws IOException {
715     Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
716         + System.currentTimeMillis());
717     if (!walFS.rename(edits, moveAsideName)) {
718       LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
719     }
720     return moveAsideName;
721   }
722 
723   private static final String SEQUENCE_ID_FILE_SUFFIX = ".seqid";
724   private static final String OLD_SEQUENCE_ID_FILE_SUFFIX = "_seqid";
725   private static final int SEQUENCE_ID_FILE_SUFFIX_LENGTH = SEQUENCE_ID_FILE_SUFFIX.length();
726 
727   /**
728    * Is the given file a region open sequence id file.
729    */
730   public static boolean isSequenceIdFile(final Path file) {
731     return file.getName().endsWith(SEQUENCE_ID_FILE_SUFFIX)
732         || file.getName().endsWith(OLD_SEQUENCE_ID_FILE_SUFFIX);
733   }
734 
735   /**
736    * Create a file with name as region open sequence id
737    * @param walFS FileSystem to write Sequence file to
738    * @param regionDir WALRegionDir used to determine where to write edits files
739    * @param newSeqId
740    * @param saftyBumper
741    * @return long new sequence Id value
742    * @throws IOException
743    */
744   public static long writeRegionSequenceIdFile(final FileSystem walFS, final Path regionDir,
745       long newSeqId, long saftyBumper) throws IOException {
746 
747     Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDir);
748     long maxSeqId = 0;
749     FileStatus[] files = null;
750     if (walFS.exists(editsdir)) {
751       files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
752         @Override
753         public boolean accept(Path p) {
754           return isSequenceIdFile(p);
755         }
756       });
757       if (files != null) {
758         for (FileStatus status : files) {
759           String fileName = status.getPath().getName();
760           try {
761             Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length()
762                 - SEQUENCE_ID_FILE_SUFFIX_LENGTH));
763             maxSeqId = Math.max(tmpSeqId, maxSeqId);
764           } catch (NumberFormatException ex) {
765             LOG.warn("Invalid SeqId File Name=" + fileName);
766           }
767         }
768       }
769     }
770     if (maxSeqId > newSeqId) {
771       newSeqId = maxSeqId;
772     }
773     newSeqId += saftyBumper; // bump up SeqId
774 
775     // write a new seqId file
776     Path newSeqIdFile = new Path(editsdir, newSeqId + SEQUENCE_ID_FILE_SUFFIX);
777     if (newSeqId != maxSeqId) {
778       try {
779         if (!walFS.createNewFile(newSeqIdFile) && !walFS.exists(newSeqIdFile)) {
780           throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
781         }
782         if (LOG.isDebugEnabled()) {
783           LOG.debug("Wrote region seqId=" + newSeqIdFile + " to file, newSeqId=" + newSeqId
784               + ", maxSeqId=" + maxSeqId);
785         }
786       } catch (FileAlreadyExistsException ignored) {
787         // latest hdfs throws this exception. it's all right if newSeqIdFile already exists
788       }
789     }
790     // remove old ones
791     if (files != null) {
792       for (FileStatus status : files) {
793         if (newSeqIdFile.equals(status.getPath())) {
794           continue;
795         }
796         walFS.delete(status.getPath(), false);
797       }
798     }
799     return newSeqId;
800   }
801 
802   /**
803    * Create a new {@link Reader} for reading logs to split.
804    *
805    * @param file
806    * @return A new Reader instance, caller should close
807    * @throws IOException
808    * @throws CorruptedLogFileException
809    */
810   protected Reader getReader(FileStatus file, boolean skipErrors, CancelableProgressable reporter)
811       throws IOException, CorruptedLogFileException {
812     Path path = file.getPath();
813     long length = file.getLen();
814     Reader in;
815 
816     // Check for possibly empty file. With appends, currently Hadoop reports a
817     // zero length even if the file has been sync'd. Revisit if HDFS-376 or
818     // HDFS-878 is committed.
819     if (length <= 0) {
820       LOG.warn("File " + path + " might be still open, length is 0");
821     }
822 
823     try {
824       FSUtils.getInstance(walFS, conf).recoverFileLease(walFS, path, conf, reporter);
825       try {
826         in = getReader(path, reporter);
827       } catch (EOFException e) {
828         if (length <= 0) {
829           // TODO should we ignore an empty, not-last log file if skip.errors
830           // is false? Either way, the caller should decide what to do. E.g.
831           // ignore if this is the last log in sequence.
832           // TODO is this scenario still possible if the log has been
833           // recovered (i.e. closed)
834           LOG.warn("Could not open " + path + " for reading. File is empty", e);
835           return null;
836         } else {
837           // EOFException being ignored
838           return null;
839         }
840       }
841     } catch (IOException e) {
842       if (e instanceof FileNotFoundException) {
843         // A wal file may not exist anymore. Nothing can be recovered so move on
844         LOG.warn("File " + path + " doesn't exist anymore.", e);
845         return null;
846       }
847       if (!skipErrors || e instanceof InterruptedIOException) {
848         throw e; // Don't mark the file corrupted if interrupted, or not skipErrors
849       }
850       throw new CorruptedLogFileException("skipErrors=true Could not open wal "
851         + path + " ignoring", e);
852     }
853     return in;
854   }
855 
856   static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
857   throws CorruptedLogFileException, IOException {
858     try {
859       return in.next();
860     } catch (EOFException eof) {
861       // truncated files are expected if a RS crashes (see HBASE-2643)
862       LOG.info("EOF from wal " + path + ".  continuing");
863       return null;
864     } catch (IOException e) {
865       // If the IOE resulted from bad file format,
866       // then this problem is idempotent and retrying won't help
867       if (e.getCause() != null &&
868           (e.getCause() instanceof ParseException ||
869            e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
870         LOG.warn("Parse exception " + e.getCause().toString() + " from wal "
871            + path + ".  continuing");
872         return null;
873       }
874       if (!skipErrors) {
875         throw e;
876       }
877       throw new CorruptedLogFileException("skipErrors=true Ignoring exception"
878         + " while parsing wal " + path + ". Marking as corrupted", e);
879     }
880   }
881 
882   /**
883    * Create a new {@link Writer} for writing log splits.
884    * @return a new Writer instance, caller should close
885    */
886   protected Writer createWriter(Path logfile)
887       throws IOException {
888     return walFactory.createRecoveredEditsWriter(walFS, logfile);
889   }
890 
891   /**
892    * Create a new {@link Reader} for reading logs to split.
893    * @return new Reader instance, caller should close
894    */
895   protected Reader getReader(Path curLogFile, CancelableProgressable reporter) throws IOException {
896     return walFactory.createReader(walFS, curLogFile, reporter);
897   }
898 
899   /**
900    * Get current open writers
901    */
902   private int getNumOpenWriters() {
903     int result = 0;
904     if (this.outputSink != null) {
905       result += this.outputSink.getNumOpenWriters();
906     }
907     return result;
908   }
909 
910   /**
911    * Contains some methods to control WAL-entries producer / consumer interactions
912    */
913   public static class PipelineController {
914     // If an exception is thrown by one of the other threads, it will be
915     // stored here.
916     AtomicReference<Throwable> thrown = new AtomicReference<Throwable>();
917 
918     // Wait/notify for when data has been produced by the writer thread,
919     // consumed by the reader thread, or an exception occurred
920     public final Object dataAvailable = new Object();
921 
922     void writerThreadError(Throwable t) {
923       thrown.compareAndSet(null, t);
924     }
925 
926     /**
927      * Check for errors in the writer threads. If any is found, rethrow it.
928      */
929     void checkForErrors() throws IOException {
930       Throwable thrown = this.thrown.get();
931       if (thrown == null) return;
932       if (thrown instanceof IOException) {
933         throw new IOException(thrown);
934       } else {
935         throw new RuntimeException(thrown);
936       }
937     }
938   }
939 
940   /**
941    * Class which accumulates edits and separates them into a buffer per region
942    * while simultaneously accounting RAM usage. Blocks if the RAM usage crosses
943    * a predefined threshold.
944    *
945    * Writer threads then pull region-specific buffers from this class.
946    */
947   public static class EntryBuffers {
948     PipelineController controller;
949 
950     Map<byte[], RegionEntryBuffer> buffers =
951       new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
952 
953     /* Track which regions are currently in the middle of writing. We don't allow
954        an IO thread to pick up bytes from a region if we're already writing
955        data for that region in a different IO thread. */
956     Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
957 
958     long totalBuffered = 0;
959     final long maxHeapUsage;
960     boolean splitWriterCreationBounded;
961 
962 
963     public EntryBuffers(PipelineController controller, long maxHeapUsage) {
964       this(controller, maxHeapUsage, false);
965     }
966 
967     public EntryBuffers(PipelineController controller, long maxHeapUsage,
968         boolean splitWriterCreationBounded) {
969       this.controller = controller;
970       this.maxHeapUsage = maxHeapUsage;
971       this.splitWriterCreationBounded = splitWriterCreationBounded;
972     }
973 
974     /**
975      * Append a log entry into the corresponding region buffer.
976      * Blocks if the total heap usage has crossed the specified threshold.
977      *
978      * @throws InterruptedException
979      * @throws IOException
980      */
981     public void appendEntry(Entry entry) throws InterruptedException, IOException {
982       WALKey key = entry.getKey();
983 
984       RegionEntryBuffer buffer;
985       long incrHeap;
986       synchronized (this) {
987         buffer = buffers.get(key.getEncodedRegionName());
988         if (buffer == null) {
989           buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
990           buffers.put(key.getEncodedRegionName(), buffer);
991         }
992         incrHeap= buffer.appendEntry(entry);
993       }
994 
995       // If we crossed the chunk threshold, wait for more space to be available
996       synchronized (controller.dataAvailable) {
997         totalBuffered += incrHeap;
998         while (totalBuffered > maxHeapUsage && controller.thrown.get() == null) {
999           LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
1000           controller.dataAvailable.wait(2000);
1001         }
1002         controller.dataAvailable.notifyAll();
1003       }
1004       controller.checkForErrors();
1005     }
1006 
1007     /**
1008      * @return RegionEntryBuffer a buffer of edits to be written or replayed.
1009      */
1010     synchronized RegionEntryBuffer getChunkToWrite() {
1011       // The core part of limiting opening writers is it doesn't return chunk only if the heap size
1012       // is over maxHeapUsage. Thus it doesn't need to create a writer for each region
1013       // during splitting. It will flush all the logs in the buffer after splitting through a
1014       // threadpool, which means the number of writers it created is under control
1015       if(splitWriterCreationBounded && totalBuffered < maxHeapUsage){
1016         return null;
1017       }
1018 
1019       long biggestSize = 0;
1020       byte[] biggestBufferKey = null;
1021 
1022       for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
1023         long size = entry.getValue().heapSize();
1024         if (size > biggestSize && (!currentlyWriting.contains(entry.getKey()))) {
1025           biggestSize = size;
1026           biggestBufferKey = entry.getKey();
1027         }
1028       }
1029       if (biggestBufferKey == null) {
1030         return null;
1031       }
1032 
1033       RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
1034       currentlyWriting.add(biggestBufferKey);
1035       return buffer;
1036     }
1037 
1038     void doneWriting(RegionEntryBuffer buffer) {
1039       synchronized (this) {
1040         boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
1041         assert removed;
1042       }
1043       long size = buffer.heapSize();
1044 
1045       synchronized (controller.dataAvailable) {
1046         totalBuffered -= size;
1047         // We may unblock writers
1048         controller.dataAvailable.notifyAll();
1049       }
1050     }
1051 
1052     synchronized boolean isRegionCurrentlyWriting(byte[] region) {
1053       return currentlyWriting.contains(region);
1054     }
1055 
1056     public void waitUntilDrained() {
1057       synchronized (controller.dataAvailable) {
1058         while (totalBuffered > 0) {
1059           try {
1060             controller.dataAvailable.wait(2000);
1061           } catch (InterruptedException e) {
1062             LOG.warn("Got interrupted while waiting for EntryBuffers is drained");
1063             Thread.interrupted();
1064             break;
1065           }
1066         }
1067       }
1068     }
1069   }
1070 
1071   /**
1072    * A buffer of some number of edits for a given region.
1073    * This accumulates edits and also provides a memory optimization in order to
1074    * share a single byte array instance for the table and region name.
1075    * Also tracks memory usage of the accumulated edits.
1076    */
1077   public static class RegionEntryBuffer implements HeapSize {
1078     long heapInBuffer = 0;
1079     List<Entry> entryBuffer;
1080     TableName tableName;
1081     byte[] encodedRegionName;
1082 
1083     RegionEntryBuffer(TableName tableName, byte[] region) {
1084       this.tableName = tableName;
1085       this.encodedRegionName = region;
1086       this.entryBuffer = new LinkedList<Entry>();
1087     }
1088 
1089     long appendEntry(Entry entry) {
1090       internify(entry);
1091       entryBuffer.add(entry);
1092       long incrHeap = entry.getEdit().heapSize() +
1093         ClassSize.align(2 * ClassSize.REFERENCE) + // WALKey pointers
1094         0; // TODO linkedlist entry
1095       heapInBuffer += incrHeap;
1096       return incrHeap;
1097     }
1098 
1099     private void internify(Entry entry) {
1100       WALKey k = entry.getKey();
1101       k.internTableName(this.tableName);
1102       k.internEncodedRegionName(this.encodedRegionName);
1103     }
1104 
1105     @Override
1106     public long heapSize() {
1107       return heapInBuffer;
1108     }
1109 
1110     public byte[] getEncodedRegionName() {
1111       return encodedRegionName;
1112     }
1113 
1114     public List<Entry> getEntryBuffer() {
1115       return entryBuffer;
1116     }
1117 
1118     public TableName getTableName() {
1119       return tableName;
1120     }
1121   }
1122 
1123   public static class WriterThread extends Thread {
1124     private volatile boolean shouldStop = false;
1125     private PipelineController controller;
1126     private EntryBuffers entryBuffers;
1127     private OutputSink outputSink = null;
1128 
1129     WriterThread(PipelineController controller, EntryBuffers entryBuffers, OutputSink sink, int i){
1130       super(Thread.currentThread().getName() + "-Writer-" + i);
1131       this.controller = controller;
1132       this.entryBuffers = entryBuffers;
1133       outputSink = sink;
1134     }
1135 
1136     @Override
1137     public void run()  {
1138       try {
1139         doRun();
1140       } catch (Throwable t) {
1141         LOG.error("Exiting thread", t);
1142         controller.writerThreadError(t);
1143       }
1144     }
1145 
1146     private void doRun() throws IOException {
1147       if (LOG.isTraceEnabled()) LOG.trace("Writer thread starting");
1148       while (true) {
1149         RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
1150         if (buffer == null) {
1151           // No data currently available, wait on some more to show up
1152           synchronized (controller.dataAvailable) {
1153             if (shouldStop && !this.outputSink.flush()) {
1154               return;
1155             }
1156             try {
1157               controller.dataAvailable.wait(500);
1158             } catch (InterruptedException ie) {
1159               if (!shouldStop) {
1160                 throw new RuntimeException(ie);
1161               }
1162             }
1163           }
1164           continue;
1165         }
1166 
1167         assert buffer != null;
1168         try {
1169           writeBuffer(buffer);
1170         } finally {
1171           entryBuffers.doneWriting(buffer);
1172         }
1173       }
1174     }
1175 
1176     private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
1177       outputSink.append(buffer);
1178     }
1179 
1180     void finish() {
1181       synchronized (controller.dataAvailable) {
1182         shouldStop = true;
1183         controller.dataAvailable.notifyAll();
1184       }
1185     }
1186   }
1187 
1188   /**
1189    * The following class is an abstraction class to provide a common interface to support both
1190    * existing recovered edits file sink and region server WAL edits replay sink
1191    */
1192   public static abstract class OutputSink {
1193 
1194     protected PipelineController controller;
1195     protected EntryBuffers entryBuffers;
1196 
1197     protected ConcurrentHashMap<String, SinkWriter> writers = new ConcurrentHashMap<>();
1198     protected ConcurrentHashMap<String, Long> regionMaximumEditLogSeqNum =
1199         new ConcurrentHashMap<>();
1200 
1201     protected final List<WriterThread> writerThreads = Lists.newArrayList();
1202 
1203     /* Set of regions which we've decided should not output edits */
1204     protected final Set<byte[]> blacklistedRegions = Collections
1205         .synchronizedSet(new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR));
1206 
1207     protected boolean closeAndCleanCompleted = false;
1208 
1209     protected boolean writersClosed = false;
1210 
1211     protected final int numThreads;
1212 
1213     protected CancelableProgressable reporter = null;
1214 
1215     protected AtomicLong skippedEdits = new AtomicLong();
1216 
1217     protected List<Path> splits = null;
1218 
1219     protected MonitoredTask status = null;
1220 
1221     public OutputSink(PipelineController controller, EntryBuffers entryBuffers, int numWriters) {
1222       numThreads = numWriters;
1223       this.controller = controller;
1224       this.entryBuffers = entryBuffers;
1225     }
1226 
1227     void setReporter(CancelableProgressable reporter) {
1228       this.reporter = reporter;
1229     }
1230 
1231     /**
1232      * Start the threads that will pump data from the entryBuffers to the output files.
1233      */
1234     public synchronized void startWriterThreads() {
1235       for (int i = 0; i < numThreads; i++) {
1236         WriterThread t = new WriterThread(controller, entryBuffers, this, i);
1237         t.start();
1238         writerThreads.add(t);
1239       }
1240     }
1241 
1242     /**
1243      *
1244      * Update region's maximum edit log SeqNum.
1245      */
1246     void updateRegionMaximumEditLogSeqNum(Entry entry) {
1247 
1248       synchronized (regionMaximumEditLogSeqNum) {
1249         String encodedRegionName = Bytes.toString(entry.getKey().getEncodedRegionName());
1250         Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(encodedRegionName);
1251         if (currentMaxSeqNum == null || entry.getKey().getLogSeqNum() > currentMaxSeqNum) {
1252           regionMaximumEditLogSeqNum.put(encodedRegionName, entry.getKey()
1253               .getLogSeqNum());
1254         }
1255       }
1256     }
1257 
1258     Long getRegionMaximumEditLogSeqNum(byte[] region) {
1259       return regionMaximumEditLogSeqNum.get(Bytes.toString(region));
1260     }
1261 
1262     /**
1263      * @return the number of currently opened writers
1264      */
1265     int getNumOpenWriters() {
1266       return this.writers.size();
1267     }
1268 
1269     long getSkippedEdits() {
1270       return this.skippedEdits.get();
1271     }
1272 
1273     void setStatus(MonitoredTask status) {
1274       this.status = status;
1275     }
1276 
1277     /**
1278      * Wait for writer threads to dump all info to the sink
1279      * @return true when there is no error
1280      * @throws IOException
1281      */
1282     protected boolean finishWriting(boolean interrupt) throws IOException {
1283       LOG.debug("Waiting for split writer threads to finish");
1284       boolean progress_failed = false;
1285       for (WriterThread t : writerThreads) {
1286         t.finish();
1287       }
1288       if (interrupt) {
1289         for (WriterThread t : writerThreads) {
1290           t.interrupt(); // interrupt the writer threads. We are stopping now.
1291         }
1292       }
1293 
1294       for (WriterThread t : writerThreads) {
1295         if (!progress_failed && reporter != null && !reporter.progress()) {
1296           progress_failed = true;
1297         }
1298         try {
1299           t.join();
1300         } catch (InterruptedException ie) {
1301           IOException iie = new InterruptedIOException();
1302           iie.initCause(ie);
1303           throw iie;
1304         }
1305       }
1306       controller.checkForErrors();
1307       final String msg = this.writerThreads.size() + " split writer threads finished";
1308       LOG.info(msg);
1309       updateStatusWithMsg(msg);
1310       return (!progress_failed);
1311     }
1312 
1313     public abstract List<Path> finishWritingAndClose() throws IOException;
1314 
1315     /**
1316      * @return a map from encoded region ID to the number of edits written out for that region.
1317      */
1318     public abstract Map<byte[], Long> getOutputCounts();
1319 
1320     /**
1321      * @return number of regions we've recovered
1322      */
1323     public abstract int getNumberOfRecoveredRegions();
1324 
1325     /**
1326      * @param buffer A WAL Edit Entry
1327      * @throws IOException
1328      */
1329     public abstract void append(RegionEntryBuffer buffer) throws IOException;
1330 
1331     /**
1332      * WriterThread call this function to help flush internal remaining edits in buffer before close
1333      * @return true when underlying sink has something to flush
1334      */
1335     public boolean flush() throws IOException {
1336       return false;
1337     }
1338 
1339     /**
1340      * Some WALEdit's contain only KV's for account on what happened to a region.
1341      * Not all sinks will want to get all of those edits.
1342      *
1343      * @return Return true if this sink wants to accept this region-level WALEdit.
1344      */
1345     public abstract boolean keepRegionEvent(Entry entry);
1346 
1347     /**
1348      * Set status message in {@link MonitoredTask} instance that is set in this OutputSink
1349      *
1350      * @param msg message to update the status with
1351      */
1352     protected final void updateStatusWithMsg(String msg) {
1353       if (status != null) {
1354         status.setStatus(msg);
1355       }
1356     }
1357   }
1358 
1359   /**
1360    * Class that manages the output streams from the log splitting process.
1361    */
1362   class LogRecoveredEditsOutputSink extends OutputSink {
1363 
1364     public LogRecoveredEditsOutputSink(PipelineController controller, EntryBuffers entryBuffers,
1365         int numWriters) {
1366       // More threads could potentially write faster at the expense
1367       // of causing more disk seeks as the logs are split.
1368       // 3. After a certain setting (probably around 3) the
1369       // process will be bound on the reader in the current
1370       // implementation anyway.
1371       super(controller, entryBuffers, numWriters);
1372     }
1373 
1374     /**
1375      * @return null if failed to report progress
1376      * @throws IOException
1377      */
1378     @Override
1379     public List<Path> finishWritingAndClose() throws IOException {
1380       boolean isSuccessful = false;
1381       List<Path> result = null;
1382       try {
1383         isSuccessful = finishWriting(false);
1384       } finally {
1385         result = close();
1386         List<IOException> thrown = closeLogWriters(null);
1387         if (thrown != null && !thrown.isEmpty()) {
1388           throw MultipleIOException.createIOException(thrown);
1389         }
1390       }
1391       if (isSuccessful) {
1392         splits = result;
1393       }
1394       return splits;
1395     }
1396 
1397     // delete the one with fewer wal entries
1398     void deleteOneWithFewerEntries(WriterAndPath wap, Path dst)
1399         throws IOException {
1400       long dstMinLogSeqNum = -1L;
1401       try (WAL.Reader reader = walFactory.createReader(walFS, dst)) {
1402         WAL.Entry entry = reader.next();
1403         if (entry != null) {
1404           dstMinLogSeqNum = entry.getKey().getLogSeqNum();
1405         }
1406       } catch (EOFException e) {
1407         if (LOG.isDebugEnabled()) {
1408           LOG.debug(
1409             "Got EOF when reading first WAL entry from " + dst + ", an empty or broken WAL file?",
1410             e);
1411         }
1412       }
1413       if (wap.minLogSeqNum < dstMinLogSeqNum) {
1414         final String errorMsg =
1415           "Found existing old edits file. It could be the result of a previous failed"
1416             + " split attempt or we have duplicated wal entries. Deleting " + dst + ", length="
1417             + walFS.getFileStatus(dst).getLen();
1418         LOG.warn(errorMsg);
1419         updateStatusWithMsg(errorMsg);
1420         if (!walFS.delete(dst, false)) {
1421           final String msg = "Failed deleting of old " + dst;
1422           LOG.warn(msg);
1423           updateStatusWithMsg(msg);
1424           throw new IOException("Failed deleting of old " + dst);
1425         }
1426       } else {
1427         final String errorMsg =
1428           "Found existing old edits file and we have less entries. Deleting " + wap.p + ", length="
1429             + walFS.getFileStatus(wap.p).getLen();
1430         LOG.warn(errorMsg);
1431         updateStatusWithMsg(errorMsg);
1432         if (!walFS.delete(wap.p, false)) {
1433           final String failureMsg = "Failed deleting of " + wap.p;
1434           LOG.warn(failureMsg);
1435           updateStatusWithMsg(failureMsg);
1436           throw new IOException(failureMsg);
1437         }
1438       }
1439     }
1440 
1441     /**
1442      * Close all of the output streams.
1443      * @return the list of paths written.
1444      */
1445     List<Path> close() throws IOException {
1446       Preconditions.checkState(!closeAndCleanCompleted);
1447 
1448       final List<Path> paths = new ArrayList<Path>();
1449       final List<IOException> thrown = Lists.newArrayList();
1450       ThreadPoolExecutor closeThreadPool = Threads.getBoundedCachedThreadPool(numThreads, 30L,
1451         TimeUnit.SECONDS, new ThreadFactory() {
1452           private int count = 1;
1453 
1454           @Override
1455           public Thread newThread(Runnable r) {
1456             Thread t = new Thread(r, "split-log-closeStream-" + count++);
1457             return t;
1458           }
1459         });
1460       CompletionService<Void> completionService =
1461         new ExecutorCompletionService<Void>(closeThreadPool);
1462       boolean progress_failed;
1463       try{
1464         progress_failed = executeCloseTask(completionService, thrown, paths);
1465       } catch (InterruptedException e) {
1466         IOException iie = new InterruptedIOException();
1467         iie.initCause(e);
1468         throw iie;
1469       } catch (ExecutionException e) {
1470         throw new IOException(e.getCause());
1471       } finally {
1472         closeThreadPool.shutdownNow();
1473       }
1474 
1475       if (!thrown.isEmpty()) {
1476         throw MultipleIOException.createIOException(thrown);
1477       }
1478       writersClosed = true;
1479       closeAndCleanCompleted = true;
1480       if (progress_failed) {
1481         return null;
1482       }
1483       return paths;
1484     }
1485 
1486     boolean executeCloseTask(CompletionService<Void> completionService,
1487         final List<IOException> thrown, final List<Path> paths)
1488         throws InterruptedException, ExecutionException {
1489       for (final Map.Entry<String, SinkWriter> writersEntry : writers.entrySet()) {
1490         if (LOG.isTraceEnabled()) {
1491           LOG.trace("Submitting close of " + ((WriterAndPath) writersEntry.getValue()).p);
1492         }
1493         completionService.submit(new Callable<Void>() {
1494           @Override public Void call() throws Exception {
1495             WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
1496             Path dst = closeWriter(writersEntry.getKey(), wap, thrown);
1497             paths.add(dst);
1498             return null;
1499           }
1500         });
1501       }
1502       boolean progress_failed = false;
1503       for (int i = 0, n = this.writers.size(); i < n; i++) {
1504         Future<Void> future = completionService.take();
1505         future.get();
1506         if (!progress_failed && reporter != null && !reporter.progress()) {
1507           progress_failed = true;
1508         }
1509       }
1510       return progress_failed;
1511     }
1512 
1513     Path closeWriter(String encodedRegionName, WriterAndPath wap, List<IOException> thrown)
1514         throws IOException {
1515       if (LOG.isTraceEnabled()) {
1516         LOG.trace("Closing " + wap.p);
1517       }
1518 
1519       try {
1520         wap.w.close();
1521       } catch (IOException ioe) {
1522         final String errorMsg = "Couldn't close log at " + wap.p;
1523         LOG.error(errorMsg, ioe);
1524         updateStatusWithMsg(errorMsg);
1525         thrown.add(ioe);
1526         return null;
1527       }
1528       final String msg =
1529         "Closed wap " + wap.p + " (wrote " + wap.editsWritten + " edits, skipped "
1530           + wap.editsSkipped + " edits in " + (wap.nanosSpent / 1000 / 1000) + "ms";
1531       if (LOG.isDebugEnabled()) {
1532         LOG.debug(msg);
1533       }
1534       updateStatusWithMsg(msg);
1535       if (wap.editsWritten == 0) {
1536         // just remove the empty recovered.edits file
1537         if (walFS.exists(wap.p) && !walFS.delete(wap.p, false)) {
1538           final String errorMsg = "Failed deleting empty " + wap.p;
1539           LOG.warn(errorMsg);
1540           throw new IOException("Failed deleting empty  " + wap.p);
1541         }
1542         return null;
1543       }
1544 
1545       Path dst = getCompletedRecoveredEditsFilePath(wap.p,
1546           regionMaximumEditLogSeqNum.get(encodedRegionName));
1547       try {
1548         if (!dst.equals(wap.p) && walFS.exists(dst)) {
1549           deleteOneWithFewerEntries(wap, dst);
1550         }
1551         // Skip the unit tests which create a splitter that reads and
1552         // writes the data without touching disk.
1553         // TestHLogSplit#testThreading is an example.
1554         if (walFS.exists(wap.p)) {
1555           if (!walFS.rename(wap.p, dst)) {
1556             final String errorMsg = "Failed renaming " + wap.p + " to " + dst;
1557             updateStatusWithMsg(errorMsg);
1558             throw new IOException(errorMsg);
1559           }
1560           final String renameLog = "Rename " + wap.p + " to " + dst;
1561           LOG.info(renameLog);
1562           updateStatusWithMsg(renameLog);
1563         }
1564       } catch (IOException ioe) {
1565         final String errorMsg = "Couldn't rename " + wap.p + " to " + dst;
1566         LOG.error(errorMsg, ioe);
1567         updateStatusWithMsg(errorMsg);
1568         thrown.add(ioe);
1569         return null;
1570       }
1571       return dst;
1572     }
1573 
1574 
1575     private List<IOException> closeLogWriters(List<IOException> thrown) throws IOException {
1576       if (writersClosed) {
1577         return thrown;
1578       }
1579 
1580       if (thrown == null) {
1581         thrown = Lists.newArrayList();
1582       }
1583       try {
1584         for (WriterThread t : writerThreads) {
1585           while (t.isAlive()) {
1586             t.shouldStop = true;
1587             t.interrupt();
1588             try {
1589               t.join(10);
1590             } catch (InterruptedException e) {
1591               IOException iie = new InterruptedIOException();
1592               iie.initCause(e);
1593               throw iie;
1594             }
1595           }
1596         }
1597       } finally {
1598         WriterAndPath wap = null;
1599         for (SinkWriter tmpWAP : writers.values()) {
1600           try {
1601             wap = (WriterAndPath) tmpWAP;
1602             wap.w.close();
1603           } catch (IOException ioe) {
1604             final String errorMsg = "Couldn't close log at " + wap.p;
1605             LOG.error(errorMsg, ioe);
1606             updateStatusWithMsg(errorMsg);
1607             thrown.add(ioe);
1608             continue;
1609           }
1610           final String msg =
1611             "Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in " + (wap.nanosSpent
1612               / 1000 / 1000) + "ms)";
1613           LOG.info(msg);
1614           updateStatusWithMsg(msg);
1615         }
1616         writersClosed = true;
1617       }
1618 
1619       return thrown;
1620     }
1621 
1622     /**
1623      * Get a writer and path for a log starting at the given entry. This function is threadsafe so
1624      * long as multiple threads are always acting on different regions.
1625      * @return null if this region shouldn't output any logs
1626      */
1627     WriterAndPath getWriterAndPath(Entry entry, boolean reusable) throws IOException {
1628       byte region[] = entry.getKey().getEncodedRegionName();
1629       String regionName = Bytes.toString(region);
1630       WriterAndPath ret = (WriterAndPath) writers.get(regionName);
1631       if (ret != null) {
1632         return ret;
1633       }
1634       // If we already decided that this region doesn't get any output
1635       // we don't need to check again.
1636       if (blacklistedRegions.contains(region)) {
1637         return null;
1638       }
1639       ret = createWAP(region, entry);
1640       if (ret == null) {
1641         blacklistedRegions.add(region);
1642         return null;
1643       }
1644 
1645       if(reusable) {
1646         writers.put(regionName, ret);
1647       }
1648       return ret;
1649     }
1650 
1651     /**
1652      * @return a path with a write for that path. caller should close.
1653      */
1654     WriterAndPath createWAP(byte[] region, Entry entry) throws IOException {
1655       String tmpDirName = conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
1656         HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
1657       Path regionedits = getRegionSplitEditsPath(entry,
1658           fileBeingSplit.getPath().getName(), tmpDirName, conf);
1659       if (regionedits == null) {
1660         return null;
1661       }
1662       if (walFS.exists(regionedits)) {
1663         final String warnMsg = "Found old edits file. It could be the "
1664           + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
1665           + walFS.getFileStatus(regionedits).getLen();
1666         LOG.warn(warnMsg);
1667         updateStatusWithMsg(warnMsg);
1668         if (!walFS.delete(regionedits, false)) {
1669           final String errorMsg = "Failed delete of old " + regionedits;
1670           LOG.warn(errorMsg);
1671           updateStatusWithMsg(errorMsg);
1672         }
1673       }
1674       Writer w = createWriter(regionedits);
1675       final String msg = "Creating writer path=" + regionedits;
1676       LOG.debug(msg);
1677       updateStatusWithMsg(msg);
1678       return new WriterAndPath(regionedits, w, entry.getKey().getLogSeqNum());
1679     }
1680 
1681     void filterCellByStore(Entry logEntry) {
1682       Map<byte[], Long> maxSeqIdInStores =
1683           regionMaxSeqIdInStores.get(Bytes.toString(logEntry.getKey().getEncodedRegionName()));
1684       if (maxSeqIdInStores == null || maxSeqIdInStores.isEmpty()) {
1685         return;
1686       }
1687       // Create the array list for the cells that aren't filtered.
1688       // We make the assumption that most cells will be kept.
1689       ArrayList<Cell> keptCells = new ArrayList<Cell>(logEntry.getEdit().getCells().size());
1690       for (Cell cell : logEntry.getEdit().getCells()) {
1691         if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
1692           keptCells.add(cell);
1693         } else {
1694           byte[] family = CellUtil.cloneFamily(cell);
1695           Long maxSeqId = maxSeqIdInStores.get(family);
1696           // Do not skip cell even if maxSeqId is null. Maybe we are in a rolling upgrade,
1697           // or the master was crashed before and we can not get the information.
1698           if (maxSeqId == null || maxSeqId.longValue() < logEntry.getKey().getLogSeqNum()) {
1699             keptCells.add(cell);
1700           }
1701         }
1702       }
1703 
1704       // Anything in the keptCells array list is still live.
1705       // So rather than removing the cells from the array list
1706       // which would be an O(n^2) operation, we just replace the list
1707       logEntry.getEdit().setCells(keptCells);
1708     }
1709 
1710     @Override
1711     public void append(RegionEntryBuffer buffer) throws IOException {
1712       appendBuffer(buffer, true);
1713     }
1714 
1715 
1716     WriterAndPath appendBuffer(RegionEntryBuffer buffer, boolean reusable) throws IOException{
1717       List<Entry> entries = buffer.entryBuffer;
1718       if (entries.isEmpty()) {
1719         LOG.warn("got an empty buffer, skipping");
1720         return null;
1721       }
1722       WriterAndPath wap = null;
1723 
1724       long startTime = System.nanoTime();
1725       int editsCount = 0;
1726 
1727       for (Entry logEntry : entries) {
1728         try {
1729           if (wap == null) {
1730             wap = getWriterAndPath(logEntry, reusable);
1731             if (wap == null) {
1732               if (LOG.isDebugEnabled()) {
1733                 LOG.debug("getWriterAndPath decided we don't need to write edits for " + logEntry);
1734               }
1735               return null;
1736             }
1737           }
1738           filterCellByStore(logEntry);
1739           if (!logEntry.getEdit().isEmpty()) {
1740             wap.w.append(logEntry);
1741             this.updateRegionMaximumEditLogSeqNum(logEntry);
1742             editsCount++;
1743           } else {
1744             wap.incrementSkippedEdits(1);
1745           }
1746         } catch (IOException e) {
1747           logAndThrowWriterAppendFailure(logEntry, e);
1748         }
1749       }
1750       // Pass along summary statistics
1751       wap.incrementEdits(editsCount);
1752       wap.incrementNanoTime(System.nanoTime() - startTime);
1753       return wap;
1754     }
1755 
1756     private void logAndThrowWriterAppendFailure(Entry logEntry, IOException e) throws IOException {
1757       e = RemoteExceptionHandler.checkIOException(e);
1758       final String errorMsg = "Failed to write log entry " + logEntry.toString() + " to log";
1759       LOG.fatal(errorMsg, e);
1760       updateStatusWithMsg(errorMsg);
1761       throw e;
1762     }
1763 
1764     @Override
1765     public boolean keepRegionEvent(Entry entry) {
1766       ArrayList<Cell> cells = entry.getEdit().getCells();
1767       for (int i = 0; i < cells.size(); i++) {
1768         if (WALEdit.isCompactionMarker(cells.get(i))) {
1769           return true;
1770         }
1771       }
1772       return false;
1773     }
1774 
1775     /**
1776      * @return a map from encoded region ID to the number of edits written out for that region.
1777      */
1778     @Override
1779     public Map<byte[], Long> getOutputCounts() {
1780       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1781       synchronized (writers) {
1782         for (Map.Entry<String, SinkWriter> entry : writers.entrySet()) {
1783           ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
1784         }
1785       }
1786       return ret;
1787     }
1788 
1789     @Override
1790     public int getNumberOfRecoveredRegions() {
1791       return writers.size();
1792     }
1793   }
1794 
1795   /**
1796    * Class that will limit the number of hdfs writers we create to split the logs
1797    */
1798   class BoundedLogWriterCreationOutputSink extends LogRecoveredEditsOutputSink{
1799 
1800     ConcurrentHashMap<String, Long> regionRecoverStatMap = new ConcurrentHashMap<>();
1801 
1802 
1803     public BoundedLogWriterCreationOutputSink(PipelineController controller,
1804         EntryBuffers entryBuffers, int numWriters){
1805       super(controller, entryBuffers, numWriters);
1806     }
1807 
1808     @Override
1809     public List<Path> finishWritingAndClose() throws IOException {
1810       boolean isSuccessful;
1811       List<Path> result;
1812       try {
1813         isSuccessful = finishWriting(false);
1814       } finally {
1815         result = close();
1816       }
1817       if (isSuccessful) {
1818         splits = result;
1819       }
1820       return splits;
1821     }
1822 
1823     @Override
1824     boolean executeCloseTask(CompletionService<Void> closeCompletionService,
1825         final List<IOException> thrown, final List<Path> paths)
1826         throws InterruptedException, ExecutionException {
1827       for (final Map.Entry<byte[], RegionEntryBuffer> buffer : entryBuffers.buffers.entrySet()) {
1828         LOG.info("Submitting write then close of " +
1829             Bytes.toString(buffer.getValue().encodedRegionName));
1830         closeCompletionService.submit(new Callable<Void>() {
1831           public Void call() throws Exception {
1832             Path dst = writeThenClose(buffer.getValue());
1833             paths.add(dst);
1834             return null;
1835           }
1836         });
1837       }
1838 
1839       boolean progress_failed = false;
1840       for (int i = 0, n = entryBuffers.buffers.size(); i < n; i++) {
1841         Future<Void> future = closeCompletionService.take();
1842         future.get();
1843         if (!progress_failed && reporter != null && !reporter.progress()) {
1844           progress_failed = true;
1845         }
1846       }
1847       return progress_failed;
1848     }
1849 
1850     @Override
1851     public Map<byte[], Long> getOutputCounts() {
1852       Map<byte[], Long> regionRecoverStatMapResult = new HashMap<>();
1853       for(Map.Entry<String, Long> entry: regionRecoverStatMap.entrySet()){
1854         regionRecoverStatMapResult.put(Bytes.toBytes(entry.getKey()), entry.getValue());
1855       }
1856       return regionRecoverStatMapResult;
1857     }
1858 
1859     @Override
1860     public int getNumberOfRecoveredRegions() {
1861       return regionRecoverStatMap.size();
1862     }
1863 
1864     @Override
1865     public void append(RegionEntryBuffer buffer) throws IOException {
1866       writeThenClose(buffer);
1867     }
1868 
1869     private Path writeThenClose(RegionEntryBuffer buffer) throws IOException {
1870       WriterAndPath wap = appendBuffer(buffer, false);
1871       Path dst = null;
1872       if(wap != null){
1873         String encodedRegionName = Bytes.toString(buffer.encodedRegionName);
1874         Long value = regionRecoverStatMap.putIfAbsent(encodedRegionName, wap.editsWritten);
1875         if(value != null){
1876           Long newValue = regionRecoverStatMap.get(encodedRegionName) + wap.editsWritten;
1877           regionRecoverStatMap.put(encodedRegionName, newValue);
1878         }
1879       }
1880 
1881       List<IOException> thrown = new ArrayList<>();
1882       if(wap != null) {
1883         dst = closeWriter(Bytes.toString(buffer.encodedRegionName), wap, thrown);
1884       }
1885 
1886       if(!thrown.isEmpty()){
1887         throw MultipleIOException.createIOException(thrown);
1888       }
1889       return dst;
1890     }
1891   }
1892 
1893 
1894   /**
1895    * Class wraps the actual writer which writes data out and related statistics
1896    */
1897   public abstract static class SinkWriter {
1898     /* Count of edits written to this path */
1899     long editsWritten = 0;
1900     /* Count of edits skipped to this path */
1901     long editsSkipped = 0;
1902     /* Number of nanos spent writing to this log */
1903     long nanosSpent = 0;
1904 
1905     void incrementEdits(int edits) {
1906       editsWritten += edits;
1907     }
1908 
1909     void incrementSkippedEdits(int skipped) {
1910       editsSkipped += skipped;
1911     }
1912 
1913     void incrementNanoTime(long nanos) {
1914       nanosSpent += nanos;
1915     }
1916   }
1917 
1918   /**
1919    * Private data structure that wraps a Writer and its Path, also collecting statistics about the
1920    * data written to this output.
1921    */
1922   private final static class WriterAndPath extends SinkWriter {
1923     final Path p;
1924     final Writer w;
1925     final long minLogSeqNum;
1926 
1927     WriterAndPath(final Path p, final Writer w, final long minLogSeqNum) {
1928       this.p = p;
1929       this.w = w;
1930       this.minLogSeqNum = minLogSeqNum;
1931     }
1932   }
1933 
1934   /**
1935    * Class that manages to replay edits from WAL files directly to assigned fail over region servers
1936    */
1937   class LogReplayOutputSink extends OutputSink {
1938     private static final double BUFFER_THRESHOLD = 0.35;
1939     private static final String KEY_DELIMITER = "#";
1940 
1941     private long waitRegionOnlineTimeOut;
1942     private final Set<String> recoveredRegions = Collections.synchronizedSet(new HashSet<String>());
1943     private final Map<String, RegionServerWriter> rsWriters =
1944         new ConcurrentHashMap<String, RegionServerWriter>();
1945     // online encoded region name -> region location map
1946     private final Map<String, HRegionLocation> onlineRegions =
1947         new ConcurrentHashMap<String, HRegionLocation>();
1948 
1949     private final Map<TableName, HConnection> tableNameToHConnectionMap = Collections
1950         .synchronizedMap(new TreeMap<TableName, HConnection>());
1951     /**
1952      * Map key -> value layout
1953      * servername:table name -> Queue(Row)
1954      */
1955     private final Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
1956         new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();
1957     private final List<Throwable> thrown = new ArrayList<Throwable>();
1958 
1959     // The following sink is used in distrubitedLogReplay mode for entries of regions in a disabling
1960     // table. It's a limitation of distributedLogReplay. Because log replay needs a region is
1961     // assigned and online before it can replay wal edits while regions of disabling/disabled table
1962     // won't be assigned by AM. We can retire this code after HBASE-8234.
1963     private LogRecoveredEditsOutputSink logRecoveredEditsOutputSink;
1964     private boolean hasEditsInDisablingOrDisabledTables = false;
1965 
1966     public LogReplayOutputSink(PipelineController controller, EntryBuffers entryBuffers,
1967         int numWriters) {
1968       super(controller, entryBuffers, numWriters);
1969       this.waitRegionOnlineTimeOut = conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT,
1970           ZKSplitLogManagerCoordination.DEFAULT_TIMEOUT);
1971       this.logRecoveredEditsOutputSink = new LogRecoveredEditsOutputSink(controller,
1972         entryBuffers, numWriters);
1973       this.logRecoveredEditsOutputSink.setReporter(reporter);
1974     }
1975 
1976     @Override
1977     public void append(RegionEntryBuffer buffer) throws IOException {
1978       List<Entry> entries = buffer.entryBuffer;
1979       if (entries.isEmpty()) {
1980         LOG.warn("got an empty buffer, skipping");
1981         return;
1982       }
1983 
1984       // check if current region in a disabling or disabled table
1985       if (disablingOrDisabledTables.contains(buffer.tableName)) {
1986         // need fall back to old way
1987         logRecoveredEditsOutputSink.append(buffer);
1988         hasEditsInDisablingOrDisabledTables = true;
1989         // store regions we have recovered so far
1990         addToRecoveredRegions(Bytes.toString(buffer.encodedRegionName));
1991         return;
1992       }
1993 
1994       // group entries by region servers
1995       groupEditsByServer(entries);
1996 
1997       // process workitems
1998       String maxLocKey = null;
1999       int maxSize = 0;
2000       List<Pair<HRegionLocation, Entry>> maxQueue = null;
2001       synchronized (this.serverToBufferQueueMap) {
2002         for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry:
2003               serverToBufferQueueMap.entrySet()) {
2004           List<Pair<HRegionLocation, Entry>> curQueue = entry.getValue();
2005           if (curQueue.size() > maxSize) {
2006             maxSize = curQueue.size();
2007             maxQueue = curQueue;
2008             maxLocKey = entry.getKey();
2009           }
2010         }
2011         if (maxSize < minBatchSize
2012             && entryBuffers.totalBuffered < BUFFER_THRESHOLD * entryBuffers.maxHeapUsage) {
2013           // buffer more to process
2014           return;
2015         } else if (maxSize > 0) {
2016           this.serverToBufferQueueMap.remove(maxLocKey);
2017         }
2018       }
2019 
2020       if (maxSize > 0) {
2021         processWorkItems(maxLocKey, maxQueue);
2022       }
2023     }
2024 
2025     private void addToRecoveredRegions(String encodedRegionName) {
2026       if (!recoveredRegions.contains(encodedRegionName)) {
2027         recoveredRegions.add(encodedRegionName);
2028       }
2029     }
2030 
2031     /**
2032      * Helper function to group WALEntries to individual region servers
2033      * @throws IOException
2034      */
2035     private void groupEditsByServer(List<Entry> entries) throws IOException {
2036       Set<TableName> nonExistentTables = null;
2037       Long cachedLastFlushedSequenceId = -1l;
2038       for (Entry entry : entries) {
2039         WALEdit edit = entry.getEdit();
2040         TableName table = entry.getKey().getTablename();
2041         // clear scopes which isn't needed for recovery
2042         entry.getKey().setScopes(null);
2043         String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
2044         // skip edits of non-existent tables
2045         if (nonExistentTables != null && nonExistentTables.contains(table)) {
2046           this.skippedEdits.incrementAndGet();
2047           continue;
2048         }
2049 
2050         Map<byte[], Long> maxStoreSequenceIds = null;
2051         boolean needSkip = false;
2052         HRegionLocation loc = null;
2053         String locKey = null;
2054         List<Cell> cells = edit.getCells();
2055         List<Cell> skippedCells = new ArrayList<Cell>();
2056         HConnection hconn = this.getConnectionByTableName(table);
2057 
2058         for (Cell cell : cells) {
2059           byte[] row = CellUtil.cloneRow(cell);
2060           byte[] family = CellUtil.cloneFamily(cell);
2061           boolean isCompactionEntry = false;
2062           if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
2063             CompactionDescriptor compaction = WALEdit.getCompaction(cell);
2064             if (compaction != null && compaction.hasRegionName()) {
2065               try {
2066                 byte[][] regionName = HRegionInfo.parseRegionName(compaction.getRegionName()
2067                   .toByteArray());
2068                 row = regionName[1]; // startKey of the region
2069                 family = compaction.getFamilyName().toByteArray();
2070                 isCompactionEntry = true;
2071               } catch (Exception ex) {
2072                 LOG.warn("Unexpected exception received, ignoring " + ex);
2073                 skippedCells.add(cell);
2074                 continue;
2075               }
2076             } else {
2077               skippedCells.add(cell);
2078               continue;
2079             }
2080           }
2081 
2082           try {
2083             loc =
2084                 locateRegionAndRefreshLastFlushedSequenceId(hconn, table, row,
2085                   encodeRegionNameStr);
2086             // skip replaying the compaction if the region is gone
2087             if (isCompactionEntry && !encodeRegionNameStr.equalsIgnoreCase(
2088               loc.getRegionInfo().getEncodedName())) {
2089               LOG.info("Not replaying a compaction marker for an older region: "
2090                   + encodeRegionNameStr);
2091               needSkip = true;
2092             }
2093           } catch (TableNotFoundException ex) {
2094             // table has been deleted so skip edits of the table
2095             LOG.info("Table " + table + " doesn't exist. Skip log replay for region "
2096                 + encodeRegionNameStr);
2097             lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE);
2098             if (nonExistentTables == null) {
2099               nonExistentTables = new TreeSet<TableName>();
2100             }
2101             nonExistentTables.add(table);
2102             this.skippedEdits.incrementAndGet();
2103             needSkip = true;
2104             break;
2105           }
2106 
2107           cachedLastFlushedSequenceId =
2108               lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
2109           if (cachedLastFlushedSequenceId != null
2110               && cachedLastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
2111             // skip the whole WAL entry
2112             this.skippedEdits.incrementAndGet();
2113             needSkip = true;
2114             break;
2115           } else {
2116             if (maxStoreSequenceIds == null) {
2117               maxStoreSequenceIds =
2118                   regionMaxSeqIdInStores.get(loc.getRegionInfo().getEncodedName());
2119             }
2120             if (maxStoreSequenceIds != null) {
2121               Long maxStoreSeqId = maxStoreSequenceIds.get(family);
2122               if (maxStoreSeqId == null || maxStoreSeqId >= entry.getKey().getLogSeqNum()) {
2123                 // skip current kv if column family doesn't exist anymore or already flushed
2124                 skippedCells.add(cell);
2125                 continue;
2126               }
2127             }
2128           }
2129         }
2130 
2131         // skip the edit
2132         if (loc == null || needSkip) continue;
2133 
2134         if (!skippedCells.isEmpty()) {
2135           cells.removeAll(skippedCells);
2136         }
2137 
2138         synchronized (serverToBufferQueueMap) {
2139           locKey = loc.getHostnamePort() + KEY_DELIMITER + table;
2140           List<Pair<HRegionLocation, Entry>> queue = serverToBufferQueueMap.get(locKey);
2141           if (queue == null) {
2142             queue =
2143                 Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, Entry>>());
2144             serverToBufferQueueMap.put(locKey, queue);
2145           }
2146           queue.add(new Pair<HRegionLocation, Entry>(loc, entry));
2147         }
2148         // store regions we have recovered so far
2149         addToRecoveredRegions(loc.getRegionInfo().getEncodedName());
2150       }
2151     }
2152 
2153     /**
2154      * Locate destination region based on table name & row. This function also makes sure the
2155      * destination region is online for replay.
2156      * @throws IOException
2157      */
2158     private HRegionLocation locateRegionAndRefreshLastFlushedSequenceId(HConnection hconn,
2159         TableName table, byte[] row, String originalEncodedRegionName) throws IOException {
2160       // fetch location from cache
2161       HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
2162       if(loc != null) return loc;
2163       // fetch location from hbase:meta directly without using cache to avoid hit old dead server
2164       loc = hconn.getRegionLocation(table, row, true);
2165       if (loc == null) {
2166         throw new IOException("Can't locate location for row:" + Bytes.toString(row)
2167             + " of table:" + table);
2168       }
2169       // check if current row moves to a different region due to region merge/split
2170       if (!originalEncodedRegionName.equalsIgnoreCase(loc.getRegionInfo().getEncodedName())) {
2171         // originalEncodedRegionName should have already flushed
2172         lastFlushedSequenceIds.put(originalEncodedRegionName, Long.MAX_VALUE);
2173         HRegionLocation tmpLoc = onlineRegions.get(loc.getRegionInfo().getEncodedName());
2174         if (tmpLoc != null) return tmpLoc;
2175       }
2176 
2177       Long lastFlushedSequenceId = -1l;
2178       AtomicBoolean isRecovering = new AtomicBoolean(true);
2179       loc = waitUntilRegionOnline(loc, row, this.waitRegionOnlineTimeOut, isRecovering);
2180       if (!isRecovering.get()) {
2181         // region isn't in recovering at all because WAL file may contain a region that has
2182         // been moved to somewhere before hosting RS fails
2183         lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), Long.MAX_VALUE);
2184         LOG.info("logReplay skip region: " + loc.getRegionInfo().getEncodedName()
2185             + " because it's not in recovering.");
2186       } else {
2187         Long cachedLastFlushedSequenceId =
2188             lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
2189 
2190         // retrieve last flushed sequence Id from ZK. Because region postOpenDeployTasks will
2191         // update the value for the region
2192         RegionStoreSequenceIds ids =
2193             csm.getSplitLogWorkerCoordination().getRegionFlushedSequenceId(failedServerName,
2194               loc.getRegionInfo().getEncodedName());
2195         if (ids != null) {
2196           lastFlushedSequenceId = ids.getLastFlushedSequenceId();
2197           Map<byte[], Long> storeIds = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2198           List<StoreSequenceId> maxSeqIdInStores = ids.getStoreSequenceIdList();
2199           for (StoreSequenceId id : maxSeqIdInStores) {
2200             storeIds.put(id.getFamilyName().toByteArray(), id.getSequenceId());
2201           }
2202           regionMaxSeqIdInStores.put(loc.getRegionInfo().getEncodedName(), storeIds);
2203         }
2204 
2205         if (cachedLastFlushedSequenceId == null
2206             || lastFlushedSequenceId > cachedLastFlushedSequenceId) {
2207           lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), lastFlushedSequenceId);
2208         }
2209       }
2210 
2211       onlineRegions.put(loc.getRegionInfo().getEncodedName(), loc);
2212       return loc;
2213     }
2214 
2215     private void processWorkItems(String key, List<Pair<HRegionLocation, Entry>> actions)
2216         throws IOException {
2217       RegionServerWriter rsw = null;
2218 
2219       long startTime = System.nanoTime();
2220       try {
2221         rsw = getRegionServerWriter(key);
2222         rsw.sink.replayEntries(actions);
2223 
2224         // Pass along summary statistics
2225         rsw.incrementEdits(actions.size());
2226         rsw.incrementNanoTime(System.nanoTime() - startTime);
2227       } catch (IOException e) {
2228         e = RemoteExceptionHandler.checkIOException(e);
2229         LOG.fatal(" Got while writing log entry to log", e);
2230         throw e;
2231       }
2232     }
2233 
2234     /**
2235      * Wait until region is online on the destination region server
2236      * @param loc
2237      * @param row
2238      * @param timeout How long to wait
2239      * @param isRecovering Recovering state of the region interested on destination region server.
2240      * @return True when region is online on the destination region server
2241      * @throws InterruptedException
2242      */
2243     private HRegionLocation waitUntilRegionOnline(HRegionLocation loc, byte[] row,
2244         final long timeout, AtomicBoolean isRecovering)
2245         throws IOException {
2246       final long endTime = EnvironmentEdgeManager.currentTime() + timeout;
2247       final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
2248         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
2249       boolean reloadLocation = false;
2250       TableName tableName = loc.getRegionInfo().getTable();
2251       int tries = 0;
2252       Throwable cause = null;
2253       while (endTime > EnvironmentEdgeManager.currentTime()) {
2254         try {
2255           // Try and get regioninfo from the hosting server.
2256           HConnection hconn = getConnectionByTableName(tableName);
2257           if(reloadLocation) {
2258             loc = hconn.getRegionLocation(tableName, row, true);
2259           }
2260           BlockingInterface remoteSvr = hconn.getAdmin(loc.getServerName());
2261           HRegionInfo region = loc.getRegionInfo();
2262           try {
2263             GetRegionInfoRequest request =
2264                 RequestConverter.buildGetRegionInfoRequest(region.getRegionName());
2265             GetRegionInfoResponse response = remoteSvr.getRegionInfo(null, request);
2266             if (HRegionInfo.convert(response.getRegionInfo()) != null) {
2267               isRecovering.set((response.hasIsRecovering()) ? response.getIsRecovering() : true);
2268               return loc;
2269             }
2270           } catch (ServiceException se) {
2271             throw ProtobufUtil.getRemoteException(se);
2272           }
2273         } catch (IOException e) {
2274           cause = e.getCause();
2275           if(!(cause instanceof RegionOpeningException)) {
2276             reloadLocation = true;
2277           }
2278         }
2279         long expectedSleep = ConnectionUtils.getPauseTime(pause, tries);
2280         try {
2281           Thread.sleep(expectedSleep);
2282         } catch (InterruptedException e) {
2283           throw new IOException("Interrupted when waiting region " +
2284               loc.getRegionInfo().getEncodedName() + " online.", e);
2285         }
2286         tries++;
2287       }
2288 
2289       throw new IOException("Timeout when waiting region " + loc.getRegionInfo().getEncodedName() +
2290         " online for " + timeout + " milliseconds.", cause);
2291     }
2292 
2293     @Override
2294     public boolean flush() throws IOException {
2295       String curLoc = null;
2296       int curSize = 0;
2297       List<Pair<HRegionLocation, Entry>> curQueue = null;
2298       synchronized (this.serverToBufferQueueMap) {
2299         for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry :
2300                 serverToBufferQueueMap.entrySet()) {
2301           String locationKey = entry.getKey();
2302           curQueue = entry.getValue();
2303           if (!curQueue.isEmpty()) {
2304             curSize = curQueue.size();
2305             curLoc = locationKey;
2306             break;
2307           }
2308         }
2309         if (curSize > 0) {
2310           this.serverToBufferQueueMap.remove(curLoc);
2311         }
2312       }
2313 
2314       if (curSize > 0) {
2315         this.processWorkItems(curLoc, curQueue);
2316         // We should already have control of the monitor; ensure this is the case.
2317         synchronized(controller.dataAvailable) {
2318           controller.dataAvailable.notifyAll();
2319         }
2320         return true;
2321       }
2322       return false;
2323     }
2324 
2325     @Override
2326     public boolean keepRegionEvent(Entry entry) {
2327       return true;
2328     }
2329 
2330     void addWriterError(Throwable t) {
2331       thrown.add(t);
2332     }
2333 
2334     @Override
2335     public List<Path> finishWritingAndClose() throws IOException {
2336       try {
2337         if (!finishWriting(false)) {
2338           return null;
2339         }
2340         if (hasEditsInDisablingOrDisabledTables) {
2341           splits = logRecoveredEditsOutputSink.finishWritingAndClose();
2342         } else {
2343           splits = new ArrayList<Path>();
2344         }
2345         // returns an empty array in order to keep interface same as old way
2346         return splits;
2347       } finally {
2348         List<IOException> thrown = closeRegionServerWriters();
2349         if (thrown != null && !thrown.isEmpty()) {
2350           throw MultipleIOException.createIOException(thrown);
2351         }
2352       }
2353     }
2354 
2355     @Override
2356     int getNumOpenWriters() {
2357       return this.rsWriters.size() + this.logRecoveredEditsOutputSink.getNumOpenWriters();
2358     }
2359 
2360     private List<IOException> closeRegionServerWriters() throws IOException {
2361       List<IOException> result = null;
2362       if (!writersClosed) {
2363         result = Lists.newArrayList();
2364         try {
2365           for (WriterThread t : writerThreads) {
2366             while (t.isAlive()) {
2367               t.shouldStop = true;
2368               t.interrupt();
2369               try {
2370                 t.join(10);
2371               } catch (InterruptedException e) {
2372                 IOException iie = new InterruptedIOException();
2373                 iie.initCause(e);
2374                 throw iie;
2375               }
2376             }
2377           }
2378         } finally {
2379           synchronized (rsWriters) {
2380             for (Map.Entry<String, RegionServerWriter> entry : rsWriters.entrySet()) {
2381               String locationKey = entry.getKey();
2382               RegionServerWriter tmpW = entry.getValue();
2383               try {
2384                 tmpW.close();
2385               } catch (IOException ioe) {
2386                 LOG.error("Couldn't close writer for region server:" + locationKey, ioe);
2387                 result.add(ioe);
2388               }
2389             }
2390           }
2391 
2392           // close connections
2393           synchronized (this.tableNameToHConnectionMap) {
2394             for (Map.Entry<TableName, HConnection> entry :
2395                   tableNameToHConnectionMap.entrySet()) {
2396               TableName tableName = entry.getKey();
2397               HConnection hconn = entry.getValue();
2398               try {
2399                 hconn.clearRegionCache();
2400                 hconn.close();
2401               } catch (IOException ioe) {
2402                 result.add(ioe);
2403               }
2404             }
2405           }
2406           writersClosed = true;
2407         }
2408       }
2409       return result;
2410     }
2411 
2412     @Override
2413     public Map<byte[], Long> getOutputCounts() {
2414       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2415       synchronized (rsWriters) {
2416         for (Map.Entry<String, RegionServerWriter> entry : rsWriters.entrySet()) {
2417           ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
2418         }
2419       }
2420       return ret;
2421     }
2422 
2423     @Override
2424     public int getNumberOfRecoveredRegions() {
2425       return this.recoveredRegions.size();
2426     }
2427 
2428     /**
2429      * Get a writer and path for a log starting at the given entry. This function is threadsafe so
2430      * long as multiple threads are always acting on different regions.
2431      * @return null if this region shouldn't output any logs
2432      */
2433     private RegionServerWriter getRegionServerWriter(String loc) throws IOException {
2434       RegionServerWriter ret = rsWriters.get(loc);
2435       if (ret != null) {
2436         return ret;
2437       }
2438 
2439       TableName tableName = getTableFromLocationStr(loc);
2440       if(tableName == null){
2441         throw new IOException("Invalid location string:" + loc + " found. Replay aborted.");
2442       }
2443 
2444       HConnection hconn = getConnectionByTableName(tableName);
2445       synchronized (rsWriters) {
2446         ret = rsWriters.get(loc);
2447         if (ret == null) {
2448           ret = new RegionServerWriter(conf, tableName, hconn);
2449           rsWriters.put(loc, ret);
2450         }
2451       }
2452       return ret;
2453     }
2454 
2455     private HConnection getConnectionByTableName(final TableName tableName) throws IOException {
2456       HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
2457       if (hconn == null) {
2458         synchronized (this.tableNameToHConnectionMap) {
2459           hconn = this.tableNameToHConnectionMap.get(tableName);
2460           if (hconn == null) {
2461             hconn = HConnectionManager.getConnection(conf);
2462             this.tableNameToHConnectionMap.put(tableName, hconn);
2463           }
2464         }
2465       }
2466       return hconn;
2467     }
2468     private TableName getTableFromLocationStr(String loc) {
2469       /**
2470        * location key is in format <server name:port>#<table name>
2471        */
2472       String[] splits = loc.split(KEY_DELIMITER);
2473       if (splits.length != 2) {
2474         return null;
2475       }
2476       return TableName.valueOf(splits[1]);
2477     }
2478   }
2479 
2480   /**
2481    * Private data structure that wraps a receiving RS and collecting statistics about the data
2482    * written to this newly assigned RS.
2483    */
2484   private final static class RegionServerWriter extends SinkWriter {
2485     final WALEditsReplaySink sink;
2486 
2487     RegionServerWriter(final Configuration conf, final TableName tableName, final HConnection conn)
2488         throws IOException {
2489       this.sink = new WALEditsReplaySink(conf, tableName, conn);
2490     }
2491 
2492     void close() throws IOException {
2493     }
2494   }
2495 
2496   static class CorruptedLogFileException extends Exception {
2497     private static final long serialVersionUID = 1L;
2498 
2499     CorruptedLogFileException(String s) {
2500       super(s);
2501     }
2502 
2503     /**
2504      * CorruptedLogFileException with cause
2505      *
2506      * @param message the message for this exception
2507      * @param cause the cause for this exception
2508      */
2509     CorruptedLogFileException(String message, Throwable cause) {
2510       super(message, cause);
2511     }
2512   }
2513 
2514   /** A struct used by getMutationsFromWALEntry */
2515   public static class MutationReplay implements Comparable<MutationReplay> {
2516     public MutationReplay(MutationType type, Mutation mutation, long nonceGroup, long nonce) {
2517       this.type = type;
2518       this.mutation = mutation;
2519       if(this.mutation.getDurability() != Durability.SKIP_WAL) {
2520         // using ASYNC_WAL for relay
2521         this.mutation.setDurability(Durability.ASYNC_WAL);
2522       }
2523       this.nonceGroup = nonceGroup;
2524       this.nonce = nonce;
2525     }
2526 
2527     public final MutationType type;
2528     public final Mutation mutation;
2529     public final long nonceGroup;
2530     public final long nonce;
2531 
2532     @Override
2533     public int compareTo(final MutationReplay d) {
2534       return this.mutation.compareTo(d.mutation);
2535     }
2536 
2537     @Override
2538     public boolean equals(Object obj) {
2539       if(!(obj instanceof MutationReplay)) {
2540         return false;
2541       } else {
2542         return this.compareTo((MutationReplay)obj) == 0;
2543       }
2544     }
2545 
2546     @Override
2547     public int hashCode() {
2548       return this.mutation.hashCode();
2549     }
2550   }
2551 
2552   /**
2553    * This function is used to construct mutations from a WALEntry. It also
2554    * reconstructs WALKey &amp; WALEdit from the passed in WALEntry
2555    * @param entry
2556    * @param cells
2557    * @param logEntry pair of WALKey and WALEdit instance stores WALKey and WALEdit instances
2558    *          extracted from the passed in WALEntry.
2559    * @param durability
2560    * @return list of Pair&lt;MutationType, Mutation&gt; to be replayed
2561    * @throws IOException
2562    */
2563   public static List<MutationReplay> getMutationsFromWALEntry(WALEntry entry, CellScanner cells,
2564       Pair<WALKey, WALEdit> logEntry, Durability durability)
2565           throws IOException {
2566     if (entry == null) {
2567       // return an empty array
2568       return new ArrayList<MutationReplay>();
2569     }
2570 
2571     long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
2572       entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
2573     int count = entry.getAssociatedCellCount();
2574     List<MutationReplay> mutations = new ArrayList<MutationReplay>();
2575     Cell previousCell = null;
2576     Mutation m = null;
2577     WALKey key = null;
2578     WALEdit val = null;
2579     if (logEntry != null) val = new WALEdit();
2580 
2581     for (int i = 0; i < count; i++) {
2582       // Throw index out of bounds if our cell count is off
2583       if (!cells.advance()) {
2584         throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
2585       }
2586       Cell cell = cells.current();
2587       if (val != null) val.add(cell);
2588 
2589       boolean isNewRowOrType =
2590           previousCell == null || previousCell.getTypeByte() != cell.getTypeByte()
2591               || !CellUtil.matchingRow(previousCell, cell);
2592       if (isNewRowOrType) {
2593         // Create new mutation
2594         if (CellUtil.isDelete(cell)) {
2595           m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2596           // Deletes don't have nonces.
2597           mutations.add(new MutationReplay(
2598               MutationType.DELETE, m, HConstants.NO_NONCE, HConstants.NO_NONCE));
2599         } else {
2600           m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2601           // Puts might come from increment or append, thus we need nonces.
2602           long nonceGroup = entry.getKey().hasNonceGroup()
2603               ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
2604           long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
2605           mutations.add(new MutationReplay(MutationType.PUT, m, nonceGroup, nonce));
2606         }
2607       }
2608       if (CellUtil.isDelete(cell)) {
2609         ((Delete) m).addDeleteMarker(cell);
2610       } else {
2611         ((Put) m).add(cell);
2612       }
2613       m.setDurability(durability);
2614       previousCell = cell;
2615     }
2616 
2617     // reconstruct WALKey
2618     if (logEntry != null) {
2619       org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKeyProto = entry.getKey();
2620       List<UUID> clusterIds = new ArrayList<UUID>(walKeyProto.getClusterIdsCount());
2621       for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
2622         clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
2623       }
2624       // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
2625       key = new HLogKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
2626               walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(),
2627               clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce(), null);
2628       logEntry.setFirst(key);
2629       logEntry.setSecond(val);
2630     }
2631 
2632     return mutations;
2633   }
2634 }