001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.wal;
019
020import java.io.EOFException;
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.io.InterruptedIOException;
024import java.text.ParseException;
025import java.util.ArrayList;
026import java.util.Arrays;
027import java.util.Collections;
028import java.util.HashMap;
029import java.util.List;
030import java.util.Map;
031import java.util.NavigableSet;
032import java.util.Set;
033import java.util.TreeMap;
034import java.util.TreeSet;
035import java.util.UUID;
036import java.util.concurrent.Callable;
037import java.util.concurrent.CompletionService;
038import java.util.concurrent.ConcurrentHashMap;
039import java.util.concurrent.ExecutionException;
040import java.util.concurrent.ExecutorCompletionService;
041import java.util.concurrent.Future;
042import java.util.concurrent.ThreadFactory;
043import java.util.concurrent.ThreadPoolExecutor;
044import java.util.concurrent.TimeUnit;
045import java.util.concurrent.atomic.AtomicLong;
046import java.util.concurrent.atomic.AtomicReference;
047import java.util.regex.Matcher;
048import java.util.regex.Pattern;
049import org.apache.commons.lang3.ArrayUtils;
050import org.apache.hadoop.conf.Configuration;
051import org.apache.hadoop.fs.FileAlreadyExistsException;
052import org.apache.hadoop.fs.FileStatus;
053import org.apache.hadoop.fs.FileSystem;
054import org.apache.hadoop.fs.Path;
055import org.apache.hadoop.fs.PathFilter;
056import org.apache.hadoop.hbase.Cell;
057import org.apache.hadoop.hbase.CellScanner;
058import org.apache.hadoop.hbase.CellUtil;
059import org.apache.hadoop.hbase.HBaseConfiguration;
060import org.apache.hadoop.hbase.HConstants;
061import org.apache.hadoop.hbase.TableName;
062import org.apache.hadoop.hbase.client.Delete;
063import org.apache.hadoop.hbase.client.Durability;
064import org.apache.hadoop.hbase.client.Mutation;
065import org.apache.hadoop.hbase.client.Put;
066import org.apache.hadoop.hbase.client.RegionInfo;
067import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
068import org.apache.hadoop.hbase.io.HeapSize;
069import org.apache.hadoop.hbase.log.HBaseMarkers;
070import org.apache.hadoop.hbase.master.SplitLogManager;
071import org.apache.hadoop.hbase.monitoring.MonitoredTask;
072import org.apache.hadoop.hbase.monitoring.TaskMonitor;
073import org.apache.hadoop.hbase.regionserver.HRegion;
074import org.apache.hadoop.hbase.regionserver.LastSequenceId;
075import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
076import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
077import org.apache.hadoop.hbase.util.Bytes;
078import org.apache.hadoop.hbase.util.CancelableProgressable;
079import org.apache.hadoop.hbase.util.ClassSize;
080import org.apache.hadoop.hbase.util.FSUtils;
081import org.apache.hadoop.hbase.util.Pair;
082import org.apache.hadoop.hbase.util.Threads;
083import org.apache.hadoop.hbase.wal.WAL.Entry;
084import org.apache.hadoop.hbase.wal.WAL.Reader;
085import org.apache.hadoop.hbase.wal.WALProvider.Writer;
086import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
087import org.apache.hadoop.io.MultipleIOException;
088import org.apache.hadoop.ipc.RemoteException;
089import org.apache.yetus.audience.InterfaceAudience;
090import org.slf4j.Logger;
091import org.slf4j.LoggerFactory;
092
093import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
094import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
095import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
096import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
097import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
098import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
099
100import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
101import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
102import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
103import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
104import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
105/**
106 * This class is responsible for splitting up a bunch of regionserver commit log
107 * files that are no longer being written to, into new files, one per region, for
108 * recovering data on startup. Delete the old log files when finished.
109 */
110@InterfaceAudience.Private
111public class WALSplitter {
112  private static final Logger LOG = LoggerFactory.getLogger(WALSplitter.class);
113
114  /** By default we retry errors in splitting, rather than skipping. */
115  public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
116
117  // Parameters for split process
118  protected final Path walDir;
119  protected final FileSystem walFS;
120  protected final Configuration conf;
121
122  // Major subcomponents of the split process.
123  // These are separated into inner classes to make testing easier.
124  OutputSink outputSink;
125  private EntryBuffers entryBuffers;
126
127  private SplitLogWorkerCoordination splitLogWorkerCoordination;
128  private final WALFactory walFactory;
129
130  private MonitoredTask status;
131
132  // For checking the latest flushed sequence id
133  protected final LastSequenceId sequenceIdChecker;
134
135  // Map encodedRegionName -> lastFlushedSequenceId
136  protected Map<String, Long> lastFlushedSequenceIds = new ConcurrentHashMap<>();
137
138  // Map encodedRegionName -> maxSeqIdInStores
139  protected Map<String, Map<byte[], Long>> regionMaxSeqIdInStores = new ConcurrentHashMap<>();
140
141  // the file being split currently
142  private FileStatus fileBeingSplit;
143
144  // if we limit the number of writers opened for sinking recovered edits
145  private final boolean splitWriterCreationBounded;
146
147  public final static String SPLIT_WRITER_CREATION_BOUNDED = "hbase.split.writer.creation.bounded";
148
149
150  @VisibleForTesting
151  WALSplitter(final WALFactory factory, Configuration conf, Path walDir,
152      FileSystem walFS, LastSequenceId idChecker,
153      SplitLogWorkerCoordination splitLogWorkerCoordination) {
154    this.conf = HBaseConfiguration.create(conf);
155    String codecClassName = conf
156        .get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
157    this.conf.set(HConstants.RPC_CODEC_CONF_KEY, codecClassName);
158    this.walDir = walDir;
159    this.walFS = walFS;
160    this.sequenceIdChecker = idChecker;
161    this.splitLogWorkerCoordination = splitLogWorkerCoordination;
162
163    this.walFactory = factory;
164    PipelineController controller = new PipelineController();
165
166    this.splitWriterCreationBounded = conf.getBoolean(SPLIT_WRITER_CREATION_BOUNDED, false);
167
168    entryBuffers = new EntryBuffers(controller,
169        this.conf.getLong("hbase.regionserver.hlog.splitlog.buffersize", 128 * 1024 * 1024),
170        splitWriterCreationBounded);
171
172    int numWriterThreads = this.conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
173    if(splitWriterCreationBounded){
174      outputSink = new BoundedLogWriterCreationOutputSink(
175          controller, entryBuffers, numWriterThreads);
176    }else {
177      outputSink = new LogRecoveredEditsOutputSink(controller, entryBuffers, numWriterThreads);
178    }
179  }
180
181  /**
182   * Splits a WAL file into region's recovered-edits directory.
183   * This is the main entry point for distributed log splitting from SplitLogWorker.
184   * <p>
185   * If the log file has N regions then N recovered.edits files will be produced.
186   * <p>
187   * @return false if it is interrupted by the progress-able.
188   */
189  public static boolean splitLogFile(Path walDir, FileStatus logfile, FileSystem walFS,
190      Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
191      SplitLogWorkerCoordination splitLogWorkerCoordination, final WALFactory factory)
192      throws IOException {
193    WALSplitter s = new WALSplitter(factory, conf, walDir, walFS, idChecker,
194        splitLogWorkerCoordination);
195    return s.splitLogFile(logfile, reporter);
196  }
197
198  // A wrapper to split one log folder using the method used by distributed
199  // log splitting. Used by tools and unit tests. It should be package private.
200  // It is public only because TestWALObserver is in a different package,
201  // which uses this method to do log splitting.
202  @VisibleForTesting
203  public static List<Path> split(Path rootDir, Path logDir, Path oldLogDir,
204      FileSystem walFS, Configuration conf, final WALFactory factory) throws IOException {
205    final FileStatus[] logfiles = SplitLogManager.getFileList(conf,
206        Collections.singletonList(logDir), null);
207    List<Path> splits = new ArrayList<>();
208    if (ArrayUtils.isNotEmpty(logfiles)) {
209      for (FileStatus logfile: logfiles) {
210        WALSplitter s = new WALSplitter(factory, conf, rootDir, walFS, null, null);
211        if (s.splitLogFile(logfile, null)) {
212          finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
213          if (s.outputSink.splits != null) {
214            splits.addAll(s.outputSink.splits);
215          }
216        }
217      }
218    }
219    if (!walFS.delete(logDir, true)) {
220      throw new IOException("Unable to delete src dir: " + logDir);
221    }
222    return splits;
223  }
224
225  /**
226   * log splitting implementation, splits one log file.
227   * @param logfile should be an actual log file.
228   */
229  @VisibleForTesting
230  boolean splitLogFile(FileStatus logfile, CancelableProgressable reporter) throws IOException {
231    Preconditions.checkState(status == null);
232    Preconditions.checkArgument(logfile.isFile(),
233        "passed in file status is for something other than a regular file.");
234    boolean isCorrupted = false;
235    boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
236      SPLIT_SKIP_ERRORS_DEFAULT);
237    int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
238    Path logPath = logfile.getPath();
239    boolean outputSinkStarted = false;
240    boolean progress_failed = false;
241    int editsCount = 0;
242    int editsSkipped = 0;
243
244    status = TaskMonitor.get().createStatus(
245          "Splitting log file " + logfile.getPath() + "into a temporary staging area.");
246    Reader logFileReader = null;
247    this.fileBeingSplit = logfile;
248    try {
249      long logLength = logfile.getLen();
250      LOG.info("Splitting WAL={}, length={}", logPath, logLength);
251      status.setStatus("Opening log file");
252      if (reporter != null && !reporter.progress()) {
253        progress_failed = true;
254        return false;
255      }
256      logFileReader = getReader(logfile, skipErrors, reporter);
257      if (logFileReader == null) {
258        LOG.warn("Nothing to split in WAL={}", logPath);
259        return true;
260      }
261      int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
262      int numOpenedFilesLastCheck = 0;
263      outputSink.setReporter(reporter);
264      outputSink.startWriterThreads();
265      outputSinkStarted = true;
266      Entry entry;
267      Long lastFlushedSequenceId = -1L;
268      while ((entry = getNextLogLine(logFileReader, logPath, skipErrors)) != null) {
269        byte[] region = entry.getKey().getEncodedRegionName();
270        String encodedRegionNameAsStr = Bytes.toString(region);
271        lastFlushedSequenceId = lastFlushedSequenceIds.get(encodedRegionNameAsStr);
272        if (lastFlushedSequenceId == null) {
273          if (sequenceIdChecker != null) {
274            RegionStoreSequenceIds ids = sequenceIdChecker.getLastSequenceId(region);
275            Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
276            for (StoreSequenceId storeSeqId : ids.getStoreSequenceIdList()) {
277              maxSeqIdInStores.put(storeSeqId.getFamilyName().toByteArray(),
278                storeSeqId.getSequenceId());
279            }
280            regionMaxSeqIdInStores.put(encodedRegionNameAsStr, maxSeqIdInStores);
281            lastFlushedSequenceId = ids.getLastFlushedSequenceId();
282            if (LOG.isDebugEnabled()) {
283              LOG.debug("DLS Last flushed sequenceid for " + encodedRegionNameAsStr + ": " +
284                  TextFormat.shortDebugString(ids));
285            }
286          }
287          if (lastFlushedSequenceId == null) {
288            lastFlushedSequenceId = -1L;
289          }
290          lastFlushedSequenceIds.put(encodedRegionNameAsStr, lastFlushedSequenceId);
291        }
292        if (lastFlushedSequenceId >= entry.getKey().getSequenceId()) {
293          editsSkipped++;
294          continue;
295        }
296        // Don't send Compaction/Close/Open region events to recovered edit type sinks.
297        if (entry.getEdit().isMetaEdit() && !outputSink.keepRegionEvent(entry)) {
298          editsSkipped++;
299          continue;
300        }
301        entryBuffers.appendEntry(entry);
302        editsCount++;
303        int moreWritersFromLastCheck = this.getNumOpenWriters() - numOpenedFilesLastCheck;
304        // If sufficient edits have passed, check if we should report progress.
305        if (editsCount % interval == 0
306            || moreWritersFromLastCheck > numOpenedFilesBeforeReporting) {
307          numOpenedFilesLastCheck = this.getNumOpenWriters();
308          String countsStr = (editsCount - (editsSkipped + outputSink.getSkippedEdits()))
309              + " edits, skipped " + editsSkipped + " edits.";
310          status.setStatus("Split " + countsStr);
311          if (reporter != null && !reporter.progress()) {
312            progress_failed = true;
313            return false;
314          }
315        }
316      }
317    } catch (InterruptedException ie) {
318      IOException iie = new InterruptedIOException();
319      iie.initCause(ie);
320      throw iie;
321    } catch (CorruptedLogFileException e) {
322      LOG.warn("Could not parse, corrupted WAL={}", logPath, e);
323      if (splitLogWorkerCoordination != null) {
324        // Some tests pass in a csm of null.
325        splitLogWorkerCoordination.markCorrupted(walDir, logfile.getPath().getName(), walFS);
326      } else {
327        // for tests only
328        ZKSplitLog.markCorrupted(walDir, logfile.getPath().getName(), walFS);
329      }
330      isCorrupted = true;
331    } catch (IOException e) {
332      e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
333      throw e;
334    } finally {
335      LOG.debug("Finishing writing output logs and closing down");
336      try {
337        if (null != logFileReader) {
338          logFileReader.close();
339        }
340      } catch (IOException exception) {
341        LOG.warn("Could not close WAL reader", exception);
342      }
343      try {
344        if (outputSinkStarted) {
345          // Set progress_failed to true as the immediate following statement will reset its value
346          // when finishWritingAndClose() throws exception, progress_failed has the right value
347          progress_failed = true;
348          progress_failed = outputSink.finishWritingAndClose() == null;
349        }
350      } finally {
351        String msg =
352            "Processed " + editsCount + " edits across " + outputSink.getNumberOfRecoveredRegions()
353                + " regions; edits skipped=" + editsSkipped + "; log file=" + logPath +
354                ", length=" + logfile.getLen() + // See if length got updated post lease recovery
355                ", corrupted=" + isCorrupted + ", progress failed=" + progress_failed;
356        LOG.info(msg);
357        status.markComplete(msg);
358      }
359    }
360    return !progress_failed;
361  }
362
363  /**
364   * Completes the work done by splitLogFile by archiving logs
365   * <p>
366   * It is invoked by SplitLogManager once it knows that one of the
367   * SplitLogWorkers have completed the splitLogFile() part. If the master
368   * crashes then this function might get called multiple times.
369   * <p>
370   * @param logfile
371   * @param conf
372   * @throws IOException
373   */
374  public static void finishSplitLogFile(String logfile,
375      Configuration conf)  throws IOException {
376    Path walDir = FSUtils.getWALRootDir(conf);
377    Path oldLogDir = new Path(walDir, HConstants.HREGION_OLDLOGDIR_NAME);
378    Path logPath;
379    if (FSUtils.isStartingWithPath(walDir, logfile)) {
380      logPath = new Path(logfile);
381    } else {
382      logPath = new Path(walDir, logfile);
383    }
384    finishSplitLogFile(walDir, oldLogDir, logPath, conf);
385  }
386
387  private static void finishSplitLogFile(Path walDir, Path oldLogDir,
388      Path logPath, Configuration conf) throws IOException {
389    List<Path> processedLogs = new ArrayList<>();
390    List<Path> corruptedLogs = new ArrayList<>();
391    FileSystem walFS = walDir.getFileSystem(conf);
392    if (ZKSplitLog.isCorrupted(walDir, logPath.getName(), walFS)) {
393      corruptedLogs.add(logPath);
394    } else {
395      processedLogs.add(logPath);
396    }
397    archiveLogs(corruptedLogs, processedLogs, oldLogDir, walFS, conf);
398    Path stagingDir = ZKSplitLog.getSplitLogDir(walDir, logPath.getName());
399    walFS.delete(stagingDir, true);
400  }
401
402  /**
403   * Moves processed logs to a oldLogDir after successful processing Moves
404   * corrupted logs (any log that couldn't be successfully parsed to corruptDir
405   * (.corrupt) for later investigation
406   *
407   * @param corruptedLogs
408   * @param processedLogs
409   * @param oldLogDir
410   * @param walFS WAL FileSystem to archive files on.
411   * @param conf
412   * @throws IOException
413   */
414  private static void archiveLogs(
415      final List<Path> corruptedLogs,
416      final List<Path> processedLogs, final Path oldLogDir,
417      final FileSystem walFS, final Configuration conf) throws IOException {
418    final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), HConstants.CORRUPT_DIR_NAME);
419    if (conf.get("hbase.regionserver.hlog.splitlog.corrupt.dir") != null) {
420      LOG.warn("hbase.regionserver.hlog.splitlog.corrupt.dir is deprecated. Default to {}",
421          corruptDir);
422    }
423    if (!walFS.mkdirs(corruptDir)) {
424      LOG.info("Unable to mkdir {}", corruptDir);
425    }
426    walFS.mkdirs(oldLogDir);
427
428    // this method can get restarted or called multiple times for archiving
429    // the same log files.
430    for (Path corrupted : corruptedLogs) {
431      Path p = new Path(corruptDir, corrupted.getName());
432      if (walFS.exists(corrupted)) {
433        if (!walFS.rename(corrupted, p)) {
434          LOG.warn("Unable to move corrupted log {} to {}", corrupted, p);
435        } else {
436          LOG.warn("Moved corrupted log {} to {}", corrupted, p);
437        }
438      }
439    }
440
441    for (Path p : processedLogs) {
442      Path newPath = AbstractFSWAL.getWALArchivePath(oldLogDir, p);
443      if (walFS.exists(p)) {
444        if (!FSUtils.renameAndSetModifyTime(walFS, p, newPath)) {
445          LOG.warn("Unable to move {} to {}", p, newPath);
446        } else {
447          LOG.info("Archived processed log {} to {}", p, newPath);
448        }
449      }
450    }
451  }
452
453  /**
454   * Path to a file under RECOVERED_EDITS_DIR directory of the region found in
455   * <code>logEntry</code> named for the sequenceid in the passed
456   * <code>logEntry</code>: e.g. /hbase/some_table/2323432434/recovered.edits/2332.
457   * This method also ensures existence of RECOVERED_EDITS_DIR under the region
458   * creating it if necessary.
459   * @param logEntry
460   * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
461   * @param tmpDirName of the directory used to sideline old recovered edits file
462   * @param conf
463   * @return Path to file into which to dump split log edits.
464   * @throws IOException
465   */
466  @SuppressWarnings("deprecation")
467  @VisibleForTesting
468  static Path getRegionSplitEditsPath(final Entry logEntry, String fileNameBeingSplit,
469      String tmpDirName, Configuration conf) throws IOException {
470    FileSystem walFS = FSUtils.getWALFileSystem(conf);
471    Path tableDir = FSUtils.getWALTableDir(conf, logEntry.getKey().getTableName());
472    String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
473    Path regionDir = HRegion.getRegionDir(tableDir, encodedRegionName);
474    Path dir = getRegionDirRecoveredEditsDir(regionDir);
475
476
477    if (walFS.exists(dir) && walFS.isFile(dir)) {
478      Path tmp = new Path(tmpDirName);
479      if (!walFS.exists(tmp)) {
480        walFS.mkdirs(tmp);
481      }
482      tmp = new Path(tmp,
483        HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
484      LOG.warn("Found existing old file: {}. It could be some "
485        + "leftover of an old installation. It should be a folder instead. "
486        + "So moving it to {}", dir, tmp);
487      if (!walFS.rename(dir, tmp)) {
488        LOG.warn("Failed to sideline old file {}", dir);
489      }
490    }
491
492    if (!walFS.exists(dir) && !walFS.mkdirs(dir)) {
493      LOG.warn("mkdir failed on {}", dir);
494    }
495    // Append fileBeingSplit to prevent name conflict since we may have duplicate wal entries now.
496    // Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
497    // region's replayRecoveredEdits will not delete it
498    String fileName = formatRecoveredEditsFileName(logEntry.getKey().getSequenceId());
499    fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileNameBeingSplit);
500    return new Path(dir, fileName);
501  }
502
503  private static String getTmpRecoveredEditsFileName(String fileName) {
504    return fileName + RECOVERED_LOG_TMPFILE_SUFFIX;
505  }
506
507  /**
508   * Get the completed recovered edits file path, renaming it to be by last edit
509   * in the file from its first edit. Then we could use the name to skip
510   * recovered edits when doing {@link HRegion#replayRecoveredEditsIfAny}.
511   * @param srcPath
512   * @param maximumEditLogSeqNum
513   * @return dstPath take file's last edit log seq num as the name
514   */
515  private static Path getCompletedRecoveredEditsFilePath(Path srcPath,
516      long maximumEditLogSeqNum) {
517    String fileName = formatRecoveredEditsFileName(maximumEditLogSeqNum);
518    return new Path(srcPath.getParent(), fileName);
519  }
520
521  @VisibleForTesting
522  static String formatRecoveredEditsFileName(final long seqid) {
523    return String.format("%019d", seqid);
524  }
525
526  private static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
527  private static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
528
529  /**
530   * @param regionDir
531   *          This regions directory in the filesystem.
532   * @return The directory that holds recovered edits files for the region
533   *         <code>regionDir</code>
534   */
535  public static Path getRegionDirRecoveredEditsDir(final Path regionDir) {
536    return new Path(regionDir, HConstants.RECOVERED_EDITS_DIR);
537  }
538
539  /**
540   * Check whether there is recovered.edits in the region dir
541   * @param walFS FileSystem
542   * @param conf conf
543   * @param regionInfo the region to check
544   * @throws IOException IOException
545   * @return true if recovered.edits exist in the region dir
546   */
547  public static boolean hasRecoveredEdits(final FileSystem walFS,
548      final Configuration conf, final RegionInfo regionInfo) throws IOException {
549    // No recovered.edits for non default replica regions
550    if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
551      return false;
552    }
553    //Only default replica region can reach here, so we can use regioninfo
554    //directly without converting it to default replica's regioninfo.
555    Path regionDir = FSUtils.getWALRegionDir(conf, regionInfo.getTable(),
556        regionInfo.getEncodedName());
557    NavigableSet<Path> files = getSplitEditFilesSorted(walFS, regionDir);
558    return files != null && !files.isEmpty();
559  }
560
561
562  /**
563   * Returns sorted set of edit files made by splitter, excluding files
564   * with '.temp' suffix.
565   *
566   * @param walFS WAL FileSystem used to retrieving split edits files.
567   * @param regionDir WAL region dir to look for recovered edits files under.
568   * @return Files in passed <code>regionDir</code> as a sorted set.
569   * @throws IOException
570   */
571  public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem walFS,
572      final Path regionDir) throws IOException {
573    NavigableSet<Path> filesSorted = new TreeSet<>();
574    Path editsdir = getRegionDirRecoveredEditsDir(regionDir);
575    if (!walFS.exists(editsdir)) {
576      return filesSorted;
577    }
578    FileStatus[] files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
579      @Override
580      public boolean accept(Path p) {
581        boolean result = false;
582        try {
583          // Return files and only files that match the editfile names pattern.
584          // There can be other files in this directory other than edit files.
585          // In particular, on error, we'll move aside the bad edit file giving
586          // it a timestamp suffix. See moveAsideBadEditsFile.
587          Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
588          result = walFS.isFile(p) && m.matches();
589          // Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
590          // because it means splitwal thread is writting this file.
591          if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
592            result = false;
593          }
594          // Skip SeqId Files
595          if (isSequenceIdFile(p)) {
596            result = false;
597          }
598        } catch (IOException e) {
599          LOG.warn("Failed isFile check on {}", p, e);
600        }
601        return result;
602      }
603    });
604    if (ArrayUtils.isNotEmpty(files)) {
605      Arrays.asList(files).forEach(status -> filesSorted.add(status.getPath()));
606    }
607    return filesSorted;
608  }
609
610  /**
611   * Move aside a bad edits file.
612   *
613   * @param walFS WAL FileSystem used to rename bad edits file.
614   * @param edits
615   *          Edits file to move aside.
616   * @return The name of the moved aside file.
617   * @throws IOException
618   */
619  public static Path moveAsideBadEditsFile(final FileSystem walFS, final Path edits)
620      throws IOException {
621    Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
622        + System.currentTimeMillis());
623    if (!walFS.rename(edits, moveAsideName)) {
624      LOG.warn("Rename failed from {} to {}", edits, moveAsideName);
625    }
626    return moveAsideName;
627  }
628
629  private static final String SEQUENCE_ID_FILE_SUFFIX = ".seqid";
630  private static final String OLD_SEQUENCE_ID_FILE_SUFFIX = "_seqid";
631  private static final int SEQUENCE_ID_FILE_SUFFIX_LENGTH = SEQUENCE_ID_FILE_SUFFIX.length();
632
633  /**
634   * Is the given file a region open sequence id file.
635   */
636  @VisibleForTesting
637  public static boolean isSequenceIdFile(final Path file) {
638    return file.getName().endsWith(SEQUENCE_ID_FILE_SUFFIX)
639        || file.getName().endsWith(OLD_SEQUENCE_ID_FILE_SUFFIX);
640  }
641
642  private static FileStatus[] getSequenceIdFiles(FileSystem walFS, Path regionDir)
643      throws IOException {
644    // TODO: Why are we using a method in here as part of our normal region open where
645    // there is no splitting involved? Fix. St.Ack 01/20/2017.
646    Path editsDir = WALSplitter.getRegionDirRecoveredEditsDir(regionDir);
647    try {
648      FileStatus[] files = walFS.listStatus(editsDir, WALSplitter::isSequenceIdFile);
649      return files != null ? files : new FileStatus[0];
650    } catch (FileNotFoundException e) {
651      return new FileStatus[0];
652    }
653  }
654
655  private static long getMaxSequenceId(FileStatus[] files) {
656    long maxSeqId = -1L;
657    for (FileStatus file : files) {
658      String fileName = file.getPath().getName();
659      try {
660        maxSeqId = Math.max(maxSeqId, Long
661          .parseLong(fileName.substring(0, fileName.length() - SEQUENCE_ID_FILE_SUFFIX_LENGTH)));
662      } catch (NumberFormatException ex) {
663        LOG.warn("Invalid SeqId File Name={}", fileName);
664      }
665    }
666    return maxSeqId;
667  }
668
669  /**
670   * Get the max sequence id which is stored in the region directory. -1 if none.
671   */
672  public static long getMaxRegionSequenceId(FileSystem walFS, Path regionDir) throws IOException {
673    return getMaxSequenceId(getSequenceIdFiles(walFS, regionDir));
674  }
675
676  /**
677   * Create a file with name as region's max sequence id
678   */
679  public static void writeRegionSequenceIdFile(FileSystem walFS, Path regionDir, long newMaxSeqId)
680      throws IOException {
681    FileStatus[] files = getSequenceIdFiles(walFS, regionDir);
682    long maxSeqId = getMaxSequenceId(files);
683    if (maxSeqId > newMaxSeqId) {
684      throw new IOException("The new max sequence id " + newMaxSeqId +
685        " is less than the old max sequence id " + maxSeqId);
686    }
687    // write a new seqId file
688    Path newSeqIdFile = new Path(WALSplitter.getRegionDirRecoveredEditsDir(regionDir),
689      newMaxSeqId + SEQUENCE_ID_FILE_SUFFIX);
690    if (newMaxSeqId != maxSeqId) {
691      try {
692        if (!walFS.createNewFile(newSeqIdFile) && !walFS.exists(newSeqIdFile)) {
693          throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
694        }
695        LOG.debug("Wrote file={}, newMaxSeqId={}, maxSeqId={}", newSeqIdFile, newMaxSeqId,
696          maxSeqId);
697      } catch (FileAlreadyExistsException ignored) {
698        // latest hdfs throws this exception. it's all right if newSeqIdFile already exists
699      }
700    }
701    // remove old ones
702    for (FileStatus status : files) {
703      if (!newSeqIdFile.equals(status.getPath())) {
704        walFS.delete(status.getPath(), false);
705      }
706    }
707  }
708
709  /**
710   * Create a new {@link Reader} for reading logs to split.
711   *
712   * @param file
713   * @return A new Reader instance, caller should close
714   * @throws IOException
715   * @throws CorruptedLogFileException
716   */
717  protected Reader getReader(FileStatus file, boolean skipErrors, CancelableProgressable reporter)
718      throws IOException, CorruptedLogFileException {
719    Path path = file.getPath();
720    long length = file.getLen();
721    Reader in;
722
723    // Check for possibly empty file. With appends, currently Hadoop reports a
724    // zero length even if the file has been sync'd. Revisit if HDFS-376 or
725    // HDFS-878 is committed.
726    if (length <= 0) {
727      LOG.warn("File {} might be still open, length is 0", path);
728    }
729
730    try {
731      FSUtils.getInstance(walFS, conf).recoverFileLease(walFS, path, conf, reporter);
732      try {
733        in = getReader(path, reporter);
734      } catch (EOFException e) {
735        if (length <= 0) {
736          // TODO should we ignore an empty, not-last log file if skip.errors
737          // is false? Either way, the caller should decide what to do. E.g.
738          // ignore if this is the last log in sequence.
739          // TODO is this scenario still possible if the log has been
740          // recovered (i.e. closed)
741          LOG.warn("Could not open {} for reading. File is empty", path, e);
742        }
743        // EOFException being ignored
744        return null;
745      }
746    } catch (IOException e) {
747      if (e instanceof FileNotFoundException) {
748        // A wal file may not exist anymore. Nothing can be recovered so move on
749        LOG.warn("File {} does not exist anymore", path, e);
750        return null;
751      }
752      if (!skipErrors || e instanceof InterruptedIOException) {
753        throw e; // Don't mark the file corrupted if interrupted, or not skipErrors
754      }
755      CorruptedLogFileException t =
756        new CorruptedLogFileException("skipErrors=true Could not open wal " +
757            path + " ignoring");
758      t.initCause(e);
759      throw t;
760    }
761    return in;
762  }
763
764  static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
765  throws CorruptedLogFileException, IOException {
766    try {
767      return in.next();
768    } catch (EOFException eof) {
769      // truncated files are expected if a RS crashes (see HBASE-2643)
770      LOG.info("EOF from wal {}. Continuing.", path);
771      return null;
772    } catch (IOException e) {
773      // If the IOE resulted from bad file format,
774      // then this problem is idempotent and retrying won't help
775      if (e.getCause() != null &&
776          (e.getCause() instanceof ParseException ||
777           e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
778        LOG.warn("Parse exception from wal {}. Continuing", path, e);
779        return null;
780      }
781      if (!skipErrors) {
782        throw e;
783      }
784      CorruptedLogFileException t =
785        new CorruptedLogFileException("skipErrors=true Ignoring exception" +
786            " while parsing wal " + path + ". Marking as corrupted");
787      t.initCause(e);
788      throw t;
789    }
790  }
791
792  /**
793   * Create a new {@link Writer} for writing log splits.
794   * @return a new Writer instance, caller should close
795   */
796  protected Writer createWriter(Path logfile)
797      throws IOException {
798    return walFactory.createRecoveredEditsWriter(walFS, logfile);
799  }
800
801  /**
802   * Create a new {@link Reader} for reading logs to split.
803   * @return new Reader instance, caller should close
804   */
805  protected Reader getReader(Path curLogFile, CancelableProgressable reporter) throws IOException {
806    return walFactory.createReader(walFS, curLogFile, reporter);
807  }
808
809  /**
810   * Get current open writers
811   */
812  private int getNumOpenWriters() {
813    int result = 0;
814    if (this.outputSink != null) {
815      result += this.outputSink.getNumOpenWriters();
816    }
817    return result;
818  }
819
820  /**
821   * Contains some methods to control WAL-entries producer / consumer interactions
822   */
823  public static class PipelineController {
824    // If an exception is thrown by one of the other threads, it will be
825    // stored here.
826    AtomicReference<Throwable> thrown = new AtomicReference<>();
827
828    // Wait/notify for when data has been produced by the writer thread,
829    // consumed by the reader thread, or an exception occurred
830    public final Object dataAvailable = new Object();
831
832    void writerThreadError(Throwable t) {
833      thrown.compareAndSet(null, t);
834    }
835
836    /**
837     * Check for errors in the writer threads. If any is found, rethrow it.
838     */
839    void checkForErrors() throws IOException {
840      Throwable thrown = this.thrown.get();
841      if (thrown == null) return;
842      if (thrown instanceof IOException) {
843        throw new IOException(thrown);
844      } else {
845        throw new RuntimeException(thrown);
846      }
847    }
848  }
849
850  /**
851   * Class which accumulates edits and separates them into a buffer per region
852   * while simultaneously accounting RAM usage. Blocks if the RAM usage crosses
853   * a predefined threshold.
854   *
855   * Writer threads then pull region-specific buffers from this class.
856   */
857  public static class EntryBuffers {
858    PipelineController controller;
859
860    Map<byte[], RegionEntryBuffer> buffers = new TreeMap<>(Bytes.BYTES_COMPARATOR);
861
862    /* Track which regions are currently in the middle of writing. We don't allow
863       an IO thread to pick up bytes from a region if we're already writing
864       data for that region in a different IO thread. */
865    Set<byte[]> currentlyWriting = new TreeSet<>(Bytes.BYTES_COMPARATOR);
866
867    long totalBuffered = 0;
868    long maxHeapUsage;
869    boolean splitWriterCreationBounded;
870
871    public EntryBuffers(PipelineController controller, long maxHeapUsage) {
872      this(controller, maxHeapUsage, false);
873    }
874
875    public EntryBuffers(PipelineController controller, long maxHeapUsage,
876        boolean splitWriterCreationBounded){
877      this.controller = controller;
878      this.maxHeapUsage = maxHeapUsage;
879      this.splitWriterCreationBounded = splitWriterCreationBounded;
880    }
881
882    /**
883     * Append a log entry into the corresponding region buffer.
884     * Blocks if the total heap usage has crossed the specified threshold.
885     *
886     * @throws InterruptedException
887     * @throws IOException
888     */
889    public void appendEntry(Entry entry) throws InterruptedException, IOException {
890      WALKey key = entry.getKey();
891
892      RegionEntryBuffer buffer;
893      long incrHeap;
894      synchronized (this) {
895        buffer = buffers.get(key.getEncodedRegionName());
896        if (buffer == null) {
897          buffer = new RegionEntryBuffer(key.getTableName(), key.getEncodedRegionName());
898          buffers.put(key.getEncodedRegionName(), buffer);
899        }
900        incrHeap= buffer.appendEntry(entry);
901      }
902
903      // If we crossed the chunk threshold, wait for more space to be available
904      synchronized (controller.dataAvailable) {
905        totalBuffered += incrHeap;
906        while (totalBuffered > maxHeapUsage && controller.thrown.get() == null) {
907          LOG.debug("Used {} bytes of buffered edits, waiting for IO threads", totalBuffered);
908          controller.dataAvailable.wait(2000);
909        }
910        controller.dataAvailable.notifyAll();
911      }
912      controller.checkForErrors();
913    }
914
915    /**
916     * @return RegionEntryBuffer a buffer of edits to be written.
917     */
918    synchronized RegionEntryBuffer getChunkToWrite() {
919      // The core part of limiting opening writers is it doesn't return chunk only if the
920      // heap size is over maxHeapUsage. Thus it doesn't need to create a writer for each
921      // region during splitting. It will flush all the logs in the buffer after splitting
922      // through a threadpool, which means the number of writers it created is under control.
923      if (splitWriterCreationBounded && totalBuffered < maxHeapUsage) {
924        return null;
925      }
926      long biggestSize = 0;
927      byte[] biggestBufferKey = null;
928
929      for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
930        long size = entry.getValue().heapSize();
931        if (size > biggestSize && (!currentlyWriting.contains(entry.getKey()))) {
932          biggestSize = size;
933          biggestBufferKey = entry.getKey();
934        }
935      }
936      if (biggestBufferKey == null) {
937        return null;
938      }
939
940      RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
941      currentlyWriting.add(biggestBufferKey);
942      return buffer;
943    }
944
945    void doneWriting(RegionEntryBuffer buffer) {
946      synchronized (this) {
947        boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
948        assert removed;
949      }
950      long size = buffer.heapSize();
951
952      synchronized (controller.dataAvailable) {
953        totalBuffered -= size;
954        // We may unblock writers
955        controller.dataAvailable.notifyAll();
956      }
957    }
958
959    synchronized boolean isRegionCurrentlyWriting(byte[] region) {
960      return currentlyWriting.contains(region);
961    }
962
963    public void waitUntilDrained() {
964      synchronized (controller.dataAvailable) {
965        while (totalBuffered > 0) {
966          try {
967            controller.dataAvailable.wait(2000);
968          } catch (InterruptedException e) {
969            LOG.warn("Got interrupted while waiting for EntryBuffers is drained");
970            Thread.interrupted();
971            break;
972          }
973        }
974      }
975    }
976  }
977
978  /**
979   * A buffer of some number of edits for a given region.
980   * This accumulates edits and also provides a memory optimization in order to
981   * share a single byte array instance for the table and region name.
982   * Also tracks memory usage of the accumulated edits.
983   */
984  public static class RegionEntryBuffer implements HeapSize {
985    long heapInBuffer = 0;
986    List<Entry> entryBuffer;
987    TableName tableName;
988    byte[] encodedRegionName;
989
990    RegionEntryBuffer(TableName tableName, byte[] region) {
991      this.tableName = tableName;
992      this.encodedRegionName = region;
993      this.entryBuffer = new ArrayList<>();
994    }
995
996    long appendEntry(Entry entry) {
997      internify(entry);
998      entryBuffer.add(entry);
999      long incrHeap = entry.getEdit().heapSize() +
1000        ClassSize.align(2 * ClassSize.REFERENCE) + // WALKey pointers
1001        0; // TODO linkedlist entry
1002      heapInBuffer += incrHeap;
1003      return incrHeap;
1004    }
1005
1006    private void internify(Entry entry) {
1007      WALKeyImpl k = entry.getKey();
1008      k.internTableName(this.tableName);
1009      k.internEncodedRegionName(this.encodedRegionName);
1010    }
1011
1012    @Override
1013    public long heapSize() {
1014      return heapInBuffer;
1015    }
1016
1017    public byte[] getEncodedRegionName() {
1018      return encodedRegionName;
1019    }
1020
1021    public List<Entry> getEntryBuffer() {
1022      return entryBuffer;
1023    }
1024
1025    public TableName getTableName() {
1026      return tableName;
1027    }
1028  }
1029
1030  public static class WriterThread extends Thread {
1031    private volatile boolean shouldStop = false;
1032    private PipelineController controller;
1033    private EntryBuffers entryBuffers;
1034    private OutputSink outputSink = null;
1035
1036    WriterThread(PipelineController controller, EntryBuffers entryBuffers, OutputSink sink, int i){
1037      super(Thread.currentThread().getName() + "-Writer-" + i);
1038      this.controller = controller;
1039      this.entryBuffers = entryBuffers;
1040      outputSink = sink;
1041    }
1042
1043    @Override
1044    public void run()  {
1045      try {
1046        doRun();
1047      } catch (Throwable t) {
1048        LOG.error("Exiting thread", t);
1049        controller.writerThreadError(t);
1050      }
1051    }
1052
1053    private void doRun() throws IOException {
1054      LOG.trace("Writer thread starting");
1055      while (true) {
1056        RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
1057        if (buffer == null) {
1058          // No data currently available, wait on some more to show up
1059          synchronized (controller.dataAvailable) {
1060            if (shouldStop && !this.outputSink.flush()) {
1061              return;
1062            }
1063            try {
1064              controller.dataAvailable.wait(500);
1065            } catch (InterruptedException ie) {
1066              if (!shouldStop) {
1067                throw new RuntimeException(ie);
1068              }
1069            }
1070          }
1071          continue;
1072        }
1073
1074        assert buffer != null;
1075        try {
1076          writeBuffer(buffer);
1077        } finally {
1078          entryBuffers.doneWriting(buffer);
1079        }
1080      }
1081    }
1082
1083    private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
1084      outputSink.append(buffer);
1085    }
1086
1087    void finish() {
1088      synchronized (controller.dataAvailable) {
1089        shouldStop = true;
1090        controller.dataAvailable.notifyAll();
1091      }
1092    }
1093  }
1094
1095  /**
1096   * The following class is an abstraction class to provide a common interface to support
1097   * different ways of consuming recovered edits.
1098   */
1099  public static abstract class OutputSink {
1100
1101    protected PipelineController controller;
1102    protected EntryBuffers entryBuffers;
1103
1104    protected ConcurrentHashMap<String, SinkWriter> writers = new ConcurrentHashMap<>();
1105    protected final ConcurrentHashMap<String, Long> regionMaximumEditLogSeqNum =
1106        new ConcurrentHashMap<>();
1107
1108
1109    protected final List<WriterThread> writerThreads = Lists.newArrayList();
1110
1111    /* Set of regions which we've decided should not output edits */
1112    protected final Set<byte[]> blacklistedRegions = Collections
1113        .synchronizedSet(new TreeSet<>(Bytes.BYTES_COMPARATOR));
1114
1115    protected boolean closeAndCleanCompleted = false;
1116
1117    protected boolean writersClosed = false;
1118
1119    protected final int numThreads;
1120
1121    protected CancelableProgressable reporter = null;
1122
1123    protected AtomicLong skippedEdits = new AtomicLong();
1124
1125    protected List<Path> splits = null;
1126
1127    public OutputSink(PipelineController controller, EntryBuffers entryBuffers, int numWriters) {
1128      numThreads = numWriters;
1129      this.controller = controller;
1130      this.entryBuffers = entryBuffers;
1131    }
1132
1133    void setReporter(CancelableProgressable reporter) {
1134      this.reporter = reporter;
1135    }
1136
1137    /**
1138     * Start the threads that will pump data from the entryBuffers to the output files.
1139     */
1140    public synchronized void startWriterThreads() {
1141      for (int i = 0; i < numThreads; i++) {
1142        WriterThread t = new WriterThread(controller, entryBuffers, this, i);
1143        t.start();
1144        writerThreads.add(t);
1145      }
1146    }
1147
1148    /**
1149     *
1150     * Update region's maximum edit log SeqNum.
1151     */
1152    void updateRegionMaximumEditLogSeqNum(Entry entry) {
1153      synchronized (regionMaximumEditLogSeqNum) {
1154        String regionName = Bytes.toString(entry.getKey().getEncodedRegionName());
1155        Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(regionName);
1156        if (currentMaxSeqNum == null || entry.getKey().getSequenceId() > currentMaxSeqNum) {
1157          regionMaximumEditLogSeqNum.put(regionName, entry.getKey().getSequenceId());
1158        }
1159      }
1160    }
1161
1162    /**
1163     * @return the number of currently opened writers
1164     */
1165    int getNumOpenWriters() {
1166      return this.writers.size();
1167    }
1168
1169    long getSkippedEdits() {
1170      return this.skippedEdits.get();
1171    }
1172
1173    /**
1174     * Wait for writer threads to dump all info to the sink
1175     * @return true when there is no error
1176     * @throws IOException
1177     */
1178    protected boolean finishWriting(boolean interrupt) throws IOException {
1179      LOG.debug("Waiting for split writer threads to finish");
1180      boolean progress_failed = false;
1181      for (WriterThread t : writerThreads) {
1182        t.finish();
1183      }
1184      if (interrupt) {
1185        for (WriterThread t : writerThreads) {
1186          t.interrupt(); // interrupt the writer threads. We are stopping now.
1187        }
1188      }
1189
1190      for (WriterThread t : writerThreads) {
1191        if (!progress_failed && reporter != null && !reporter.progress()) {
1192          progress_failed = true;
1193        }
1194        try {
1195          t.join();
1196        } catch (InterruptedException ie) {
1197          IOException iie = new InterruptedIOException();
1198          iie.initCause(ie);
1199          throw iie;
1200        }
1201      }
1202      controller.checkForErrors();
1203      LOG.info("{} split writers finished; closing.", this.writerThreads.size());
1204      return (!progress_failed);
1205    }
1206
1207    public abstract List<Path> finishWritingAndClose() throws IOException;
1208
1209    /**
1210     * @return a map from encoded region ID to the number of edits written out for that region.
1211     */
1212    public abstract Map<byte[], Long> getOutputCounts();
1213
1214    /**
1215     * @return number of regions we've recovered
1216     */
1217    public abstract int getNumberOfRecoveredRegions();
1218
1219    /**
1220     * @param buffer A WAL Edit Entry
1221     * @throws IOException
1222     */
1223    public abstract void append(RegionEntryBuffer buffer) throws IOException;
1224
1225    /**
1226     * WriterThread call this function to help flush internal remaining edits in buffer before close
1227     * @return true when underlying sink has something to flush
1228     */
1229    public boolean flush() throws IOException {
1230      return false;
1231    }
1232
1233    /**
1234     * Some WALEdit's contain only KV's for account on what happened to a region.
1235     * Not all sinks will want to get all of those edits.
1236     *
1237     * @return Return true if this sink wants to accept this region-level WALEdit.
1238     */
1239    public abstract boolean keepRegionEvent(Entry entry);
1240  }
1241
1242  /**
1243   * Class that manages the output streams from the log splitting process.
1244   */
1245  class LogRecoveredEditsOutputSink extends OutputSink {
1246
1247    public LogRecoveredEditsOutputSink(PipelineController controller, EntryBuffers entryBuffers,
1248        int numWriters) {
1249      // More threads could potentially write faster at the expense
1250      // of causing more disk seeks as the logs are split.
1251      // 3. After a certain setting (probably around 3) the
1252      // process will be bound on the reader in the current
1253      // implementation anyway.
1254      super(controller, entryBuffers, numWriters);
1255    }
1256
1257    /**
1258     * @return null if failed to report progress
1259     * @throws IOException
1260     */
1261    @Override
1262    public List<Path> finishWritingAndClose() throws IOException {
1263      boolean isSuccessful = false;
1264      List<Path> result = null;
1265      try {
1266        isSuccessful = finishWriting(false);
1267      } finally {
1268        result = close();
1269        List<IOException> thrown = closeLogWriters(null);
1270        if (CollectionUtils.isNotEmpty(thrown)) {
1271          throw MultipleIOException.createIOException(thrown);
1272        }
1273      }
1274      if (isSuccessful) {
1275        splits = result;
1276      }
1277      return splits;
1278    }
1279
1280    // delete the one with fewer wal entries
1281    private void deleteOneWithFewerEntries(WriterAndPath wap, Path dst)
1282        throws IOException {
1283      long dstMinLogSeqNum = -1L;
1284      try (WAL.Reader reader = walFactory.createReader(walFS, dst)) {
1285        WAL.Entry entry = reader.next();
1286        if (entry != null) {
1287          dstMinLogSeqNum = entry.getKey().getSequenceId();
1288        }
1289      } catch (EOFException e) {
1290        LOG.debug("Got EOF when reading first WAL entry from {}, an empty or broken WAL file?",
1291            dst, e);
1292      }
1293      if (wap.minLogSeqNum < dstMinLogSeqNum) {
1294        LOG.warn("Found existing old edits file. It could be the result of a previous failed"
1295            + " split attempt or we have duplicated wal entries. Deleting " + dst + ", length="
1296            + walFS.getFileStatus(dst).getLen());
1297        if (!walFS.delete(dst, false)) {
1298          LOG.warn("Failed deleting of old {}", dst);
1299          throw new IOException("Failed deleting of old " + dst);
1300        }
1301      } else {
1302        LOG.warn("Found existing old edits file and we have less entries. Deleting " + wap.p
1303            + ", length=" + walFS.getFileStatus(wap.p).getLen());
1304        if (!walFS.delete(wap.p, false)) {
1305          LOG.warn("Failed deleting of {}", wap.p);
1306          throw new IOException("Failed deleting of " + wap.p);
1307        }
1308      }
1309    }
1310
1311    /**
1312     * Close all of the output streams.
1313     * @return the list of paths written.
1314     */
1315    List<Path> close() throws IOException {
1316      Preconditions.checkState(!closeAndCleanCompleted);
1317
1318      final List<Path> paths = new ArrayList<>();
1319      final List<IOException> thrown = Lists.newArrayList();
1320      ThreadPoolExecutor closeThreadPool = Threads
1321          .getBoundedCachedThreadPool(numThreads, 30L, TimeUnit.SECONDS, new ThreadFactory() {
1322            private int count = 1;
1323
1324            @Override public Thread newThread(Runnable r) {
1325              Thread t = new Thread(r, "split-log-closeStream-" + count++);
1326              return t;
1327            }
1328          });
1329      CompletionService<Void> completionService = new ExecutorCompletionService<>(closeThreadPool);
1330      boolean progress_failed;
1331      try {
1332        progress_failed = executeCloseTask(completionService, thrown, paths);
1333      } catch (InterruptedException e) {
1334        IOException iie = new InterruptedIOException();
1335        iie.initCause(e);
1336        throw iie;
1337      } catch (ExecutionException e) {
1338        throw new IOException(e.getCause());
1339      } finally {
1340        closeThreadPool.shutdownNow();
1341      }
1342      if (!thrown.isEmpty()) {
1343        throw MultipleIOException.createIOException(thrown);
1344      }
1345      writersClosed = true;
1346      closeAndCleanCompleted = true;
1347      if (progress_failed) {
1348        return null;
1349      }
1350      return paths;
1351    }
1352
1353    /**
1354     * @param completionService threadPool to execute the closing tasks
1355     * @param thrown store the exceptions
1356     * @param paths arrayList to store the paths written
1357     * @return if close tasks executed successful
1358     */
1359    boolean executeCloseTask(CompletionService<Void> completionService,
1360        List<IOException> thrown, List<Path> paths)
1361        throws InterruptedException, ExecutionException {
1362      for (final Map.Entry<String, SinkWriter> writersEntry : writers.entrySet()) {
1363        if (LOG.isTraceEnabled()) {
1364          LOG.trace("Submitting close of " + ((WriterAndPath) writersEntry.getValue()).p);
1365        }
1366        completionService.submit(new Callable<Void>() {
1367          @Override public Void call() throws Exception {
1368            WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
1369            Path dst = closeWriter(writersEntry.getKey(), wap, thrown);
1370            paths.add(dst);
1371            return null;
1372          }
1373        });
1374      }
1375      boolean progress_failed = false;
1376      for (int i = 0, n = this.writers.size(); i < n; i++) {
1377        Future<Void> future = completionService.take();
1378        future.get();
1379        if (!progress_failed && reporter != null && !reporter.progress()) {
1380          progress_failed = true;
1381        }
1382      }
1383      return progress_failed;
1384    }
1385
1386    Path closeWriter(String encodedRegionName, WriterAndPath wap,
1387        List<IOException> thrown) throws IOException{
1388      LOG.trace("Closing " + wap.p);
1389      try {
1390        wap.w.close();
1391      } catch (IOException ioe) {
1392        LOG.error("Couldn't close log at " + wap.p, ioe);
1393        thrown.add(ioe);
1394        return null;
1395      }
1396      if (LOG.isDebugEnabled()) {
1397        LOG.debug("Closed wap " + wap.p + " (wrote " + wap.editsWritten
1398            + " edits, skipped " + wap.editsSkipped + " edits in "
1399            + (wap.nanosSpent / 1000 / 1000) + "ms");
1400      }
1401      if (wap.editsWritten == 0) {
1402        // just remove the empty recovered.edits file
1403        if (walFS.exists(wap.p) && !walFS.delete(wap.p, false)) {
1404          LOG.warn("Failed deleting empty " + wap.p);
1405          throw new IOException("Failed deleting empty  " + wap.p);
1406        }
1407        return null;
1408      }
1409
1410      Path dst = getCompletedRecoveredEditsFilePath(wap.p,
1411          regionMaximumEditLogSeqNum.get(encodedRegionName));
1412      try {
1413        if (!dst.equals(wap.p) && walFS.exists(dst)) {
1414          deleteOneWithFewerEntries(wap, dst);
1415        }
1416        // Skip the unit tests which create a splitter that reads and
1417        // writes the data without touching disk.
1418        // TestHLogSplit#testThreading is an example.
1419        if (walFS.exists(wap.p)) {
1420          if (!walFS.rename(wap.p, dst)) {
1421            throw new IOException("Failed renaming " + wap.p + " to " + dst);
1422          }
1423          LOG.info("Rename " + wap.p + " to " + dst);
1424        }
1425      } catch (IOException ioe) {
1426        LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
1427        thrown.add(ioe);
1428        return null;
1429      }
1430      return dst;
1431    }
1432
1433    private List<IOException> closeLogWriters(List<IOException> thrown) throws IOException {
1434      if (writersClosed) {
1435        return thrown;
1436      }
1437      if (thrown == null) {
1438        thrown = Lists.newArrayList();
1439      }
1440      try {
1441        for (WriterThread t : writerThreads) {
1442          while (t.isAlive()) {
1443            t.shouldStop = true;
1444            t.interrupt();
1445            try {
1446              t.join(10);
1447            } catch (InterruptedException e) {
1448              IOException iie = new InterruptedIOException();
1449              iie.initCause(e);
1450              throw iie;
1451            }
1452          }
1453        }
1454      } finally {
1455        WriterAndPath wap = null;
1456        for (SinkWriter tmpWAP : writers.values()) {
1457          try {
1458            wap = (WriterAndPath) tmpWAP;
1459            wap.w.close();
1460          } catch (IOException ioe) {
1461            LOG.error("Couldn't close log at " + wap.p, ioe);
1462            thrown.add(ioe);
1463            continue;
1464          }
1465          LOG.info(
1466              "Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in " + (wap.nanosSpent
1467                  / 1000 / 1000) + "ms)");
1468        }
1469        writersClosed = true;
1470      }
1471
1472      return thrown;
1473    }
1474
1475    /**
1476     * Get a writer and path for a log starting at the given entry. This function is threadsafe so
1477     * long as multiple threads are always acting on different regions.
1478     * @return null if this region shouldn't output any logs
1479     */
1480    WriterAndPath getWriterAndPath(Entry entry, boolean reusable) throws IOException {
1481      byte region[] = entry.getKey().getEncodedRegionName();
1482      String regionName = Bytes.toString(region);
1483      WriterAndPath ret = (WriterAndPath) writers.get(regionName);
1484      if (ret != null) {
1485        return ret;
1486      }
1487      // If we already decided that this region doesn't get any output
1488      // we don't need to check again.
1489      if (blacklistedRegions.contains(region)) {
1490        return null;
1491      }
1492      ret = createWAP(region, entry);
1493      if (ret == null) {
1494        blacklistedRegions.add(region);
1495        return null;
1496      }
1497      if(reusable) {
1498        writers.put(regionName, ret);
1499      }
1500      return ret;
1501    }
1502
1503    /**
1504     * @return a path with a write for that path. caller should close.
1505     */
1506    WriterAndPath createWAP(byte[] region, Entry entry) throws IOException {
1507      String tmpDirName = conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
1508        HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
1509      Path regionedits = getRegionSplitEditsPath(entry,
1510          fileBeingSplit.getPath().getName(), tmpDirName, conf);
1511      if (regionedits == null) {
1512        return null;
1513      }
1514      FileSystem walFs = FSUtils.getWALFileSystem(conf);
1515      if (walFs.exists(regionedits)) {
1516        LOG.warn("Found old edits file. It could be the "
1517            + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
1518            + walFs.getFileStatus(regionedits).getLen());
1519        if (!walFs.delete(regionedits, false)) {
1520          LOG.warn("Failed delete of old {}", regionedits);
1521        }
1522      }
1523      Writer w = createWriter(regionedits);
1524      LOG.debug("Creating writer path={}", regionedits);
1525      return new WriterAndPath(regionedits, w, entry.getKey().getSequenceId());
1526    }
1527
1528    void filterCellByStore(Entry logEntry) {
1529      Map<byte[], Long> maxSeqIdInStores =
1530          regionMaxSeqIdInStores.get(Bytes.toString(logEntry.getKey().getEncodedRegionName()));
1531      if (MapUtils.isEmpty(maxSeqIdInStores)) {
1532        return;
1533      }
1534      // Create the array list for the cells that aren't filtered.
1535      // We make the assumption that most cells will be kept.
1536      ArrayList<Cell> keptCells = new ArrayList<>(logEntry.getEdit().getCells().size());
1537      for (Cell cell : logEntry.getEdit().getCells()) {
1538        if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
1539          keptCells.add(cell);
1540        } else {
1541          byte[] family = CellUtil.cloneFamily(cell);
1542          Long maxSeqId = maxSeqIdInStores.get(family);
1543          // Do not skip cell even if maxSeqId is null. Maybe we are in a rolling upgrade,
1544          // or the master was crashed before and we can not get the information.
1545          if (maxSeqId == null || maxSeqId.longValue() < logEntry.getKey().getSequenceId()) {
1546            keptCells.add(cell);
1547          }
1548        }
1549      }
1550
1551      // Anything in the keptCells array list is still live.
1552      // So rather than removing the cells from the array list
1553      // which would be an O(n^2) operation, we just replace the list
1554      logEntry.getEdit().setCells(keptCells);
1555    }
1556
1557    @Override
1558    public void append(RegionEntryBuffer buffer) throws IOException {
1559      appendBuffer(buffer, true);
1560    }
1561
1562    WriterAndPath appendBuffer(RegionEntryBuffer buffer, boolean reusable) throws IOException{
1563      List<Entry> entries = buffer.entryBuffer;
1564      if (entries.isEmpty()) {
1565        LOG.warn("got an empty buffer, skipping");
1566        return null;
1567      }
1568
1569      WriterAndPath wap = null;
1570
1571      long startTime = System.nanoTime();
1572      try {
1573        int editsCount = 0;
1574
1575        for (Entry logEntry : entries) {
1576          if (wap == null) {
1577            wap = getWriterAndPath(logEntry, reusable);
1578            if (wap == null) {
1579              if (LOG.isTraceEnabled()) {
1580                // This log spews the full edit. Can be massive in the log. Enable only debugging
1581                // WAL lost edit issues.
1582                LOG.trace("getWriterAndPath decided we don't need to write edits for {}", logEntry);
1583              }
1584              return null;
1585            }
1586          }
1587          filterCellByStore(logEntry);
1588          if (!logEntry.getEdit().isEmpty()) {
1589            wap.w.append(logEntry);
1590            this.updateRegionMaximumEditLogSeqNum(logEntry);
1591            editsCount++;
1592          } else {
1593            wap.incrementSkippedEdits(1);
1594          }
1595        }
1596        // Pass along summary statistics
1597        wap.incrementEdits(editsCount);
1598        wap.incrementNanoTime(System.nanoTime() - startTime);
1599      } catch (IOException e) {
1600          e = e instanceof RemoteException ?
1601                  ((RemoteException)e).unwrapRemoteException() : e;
1602        LOG.error(HBaseMarkers.FATAL, "Got while writing log entry to log", e);
1603        throw e;
1604      }
1605      return wap;
1606    }
1607
1608    @Override
1609    public boolean keepRegionEvent(Entry entry) {
1610      ArrayList<Cell> cells = entry.getEdit().getCells();
1611      for (Cell cell : cells) {
1612        if (WALEdit.isCompactionMarker(cell)) {
1613          return true;
1614        }
1615      }
1616      return false;
1617    }
1618
1619    /**
1620     * @return a map from encoded region ID to the number of edits written out for that region.
1621     */
1622    @Override
1623    public Map<byte[], Long> getOutputCounts() {
1624      TreeMap<byte[], Long> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1625      for (Map.Entry<String, SinkWriter> entry : writers.entrySet()) {
1626        ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
1627      }
1628      return ret;
1629    }
1630
1631    @Override
1632    public int getNumberOfRecoveredRegions() {
1633      return writers.size();
1634    }
1635  }
1636
1637  /**
1638   *
1639   */
1640  class BoundedLogWriterCreationOutputSink extends LogRecoveredEditsOutputSink {
1641
1642    private ConcurrentHashMap<String, Long> regionRecoverStatMap = new ConcurrentHashMap<>();
1643
1644    public BoundedLogWriterCreationOutputSink(PipelineController controller,
1645        EntryBuffers entryBuffers, int numWriters) {
1646      super(controller, entryBuffers, numWriters);
1647    }
1648
1649    @Override
1650    public List<Path> finishWritingAndClose() throws IOException {
1651      boolean isSuccessful;
1652      List<Path> result;
1653      try {
1654        isSuccessful = finishWriting(false);
1655      } finally {
1656        result = close();
1657      }
1658      if (isSuccessful) {
1659        splits = result;
1660      }
1661      return splits;
1662    }
1663
1664    @Override
1665    boolean executeCloseTask(CompletionService<Void> completionService,
1666        List<IOException> thrown, List<Path> paths)
1667        throws InterruptedException, ExecutionException {
1668      for (final Map.Entry<byte[], RegionEntryBuffer> buffer : entryBuffers.buffers.entrySet()) {
1669        LOG.info("Submitting writeThenClose of {}",
1670            Arrays.toString(buffer.getValue().encodedRegionName));
1671        completionService.submit(new Callable<Void>() {
1672          @Override
1673          public Void call() throws Exception {
1674            Path dst = writeThenClose(buffer.getValue());
1675            paths.add(dst);
1676            return null;
1677          }
1678        });
1679      }
1680      boolean progress_failed = false;
1681      for (int i = 0, n = entryBuffers.buffers.size(); i < n; i++) {
1682        Future<Void> future = completionService.take();
1683        future.get();
1684        if (!progress_failed && reporter != null && !reporter.progress()) {
1685          progress_failed = true;
1686        }
1687      }
1688
1689      return progress_failed;
1690    }
1691
1692    /**
1693     * since the splitting process may create multiple output files, we need a map
1694     * regionRecoverStatMap to track the output count of each region.
1695     * @return a map from encoded region ID to the number of edits written out for that region.
1696     */
1697    @Override
1698    public Map<byte[], Long> getOutputCounts() {
1699      Map<byte[], Long> regionRecoverStatMapResult = new HashMap<>();
1700      for(Map.Entry<String, Long> entry: regionRecoverStatMap.entrySet()){
1701        regionRecoverStatMapResult.put(Bytes.toBytes(entry.getKey()), entry.getValue());
1702      }
1703      return regionRecoverStatMapResult;
1704    }
1705
1706    /**
1707     * @return the number of recovered regions
1708     */
1709    @Override
1710    public int getNumberOfRecoveredRegions() {
1711      return regionRecoverStatMap.size();
1712    }
1713
1714    /**
1715     * Append the buffer to a new recovered edits file, then close it after all done
1716     * @param buffer contain all entries of a certain region
1717     * @throws IOException when closeWriter failed
1718     */
1719    @Override
1720    public void append(RegionEntryBuffer buffer) throws IOException {
1721      writeThenClose(buffer);
1722    }
1723
1724    private Path writeThenClose(RegionEntryBuffer buffer) throws IOException {
1725      WriterAndPath wap = appendBuffer(buffer, false);
1726      if(wap != null) {
1727        String encodedRegionName = Bytes.toString(buffer.encodedRegionName);
1728        Long value = regionRecoverStatMap.putIfAbsent(encodedRegionName, wap.editsWritten);
1729        if (value != null) {
1730          Long newValue = regionRecoverStatMap.get(encodedRegionName) + wap.editsWritten;
1731          regionRecoverStatMap.put(encodedRegionName, newValue);
1732        }
1733      }
1734
1735      Path dst = null;
1736      List<IOException> thrown = new ArrayList<>();
1737      if(wap != null){
1738        dst = closeWriter(Bytes.toString(buffer.encodedRegionName), wap, thrown);
1739      }
1740      if (!thrown.isEmpty()) {
1741        throw MultipleIOException.createIOException(thrown);
1742      }
1743      return dst;
1744    }
1745  }
1746
1747  /**
1748   * Class wraps the actual writer which writes data out and related statistics
1749   */
1750  public abstract static class SinkWriter {
1751    /* Count of edits written to this path */
1752    long editsWritten = 0;
1753    /* Count of edits skipped to this path */
1754    long editsSkipped = 0;
1755    /* Number of nanos spent writing to this log */
1756    long nanosSpent = 0;
1757
1758    void incrementEdits(int edits) {
1759      editsWritten += edits;
1760    }
1761
1762    void incrementSkippedEdits(int skipped) {
1763      editsSkipped += skipped;
1764    }
1765
1766    void incrementNanoTime(long nanos) {
1767      nanosSpent += nanos;
1768    }
1769  }
1770
1771  /**
1772   * Private data structure that wraps a Writer and its Path, also collecting statistics about the
1773   * data written to this output.
1774   */
1775  private final static class WriterAndPath extends SinkWriter {
1776    final Path p;
1777    final Writer w;
1778    final long minLogSeqNum;
1779
1780    WriterAndPath(final Path p, final Writer w, final long minLogSeqNum) {
1781      this.p = p;
1782      this.w = w;
1783      this.minLogSeqNum = minLogSeqNum;
1784    }
1785  }
1786
1787  static class CorruptedLogFileException extends Exception {
1788    private static final long serialVersionUID = 1L;
1789
1790    CorruptedLogFileException(String s) {
1791      super(s);
1792    }
1793  }
1794
1795  /** A struct used by getMutationsFromWALEntry */
1796  public static class MutationReplay implements Comparable<MutationReplay> {
1797    public MutationReplay(MutationType type, Mutation mutation, long nonceGroup, long nonce) {
1798      this.type = type;
1799      this.mutation = mutation;
1800      if(this.mutation.getDurability() != Durability.SKIP_WAL) {
1801        // using ASYNC_WAL for relay
1802        this.mutation.setDurability(Durability.ASYNC_WAL);
1803      }
1804      this.nonceGroup = nonceGroup;
1805      this.nonce = nonce;
1806    }
1807
1808    public final MutationType type;
1809    public final Mutation mutation;
1810    public final long nonceGroup;
1811    public final long nonce;
1812
1813    @Override
1814    public int compareTo(final MutationReplay d) {
1815      return this.mutation.compareTo(d.mutation);
1816    }
1817
1818    @Override
1819    public boolean equals(Object obj) {
1820      if(!(obj instanceof MutationReplay)) {
1821        return false;
1822      } else {
1823        return this.compareTo((MutationReplay)obj) == 0;
1824      }
1825    }
1826
1827    @Override
1828    public int hashCode() {
1829      return this.mutation.hashCode();
1830    }
1831  }
1832
1833  /**
1834   * This function is used to construct mutations from a WALEntry. It also
1835   * reconstructs WALKey &amp; WALEdit from the passed in WALEntry
1836   * @param entry
1837   * @param cells
1838   * @param logEntry pair of WALKey and WALEdit instance stores WALKey and WALEdit instances
1839   *          extracted from the passed in WALEntry.
1840   * @return list of Pair&lt;MutationType, Mutation&gt; to be replayed
1841   * @throws IOException
1842   */
1843  public static List<MutationReplay> getMutationsFromWALEntry(WALEntry entry, CellScanner cells,
1844      Pair<WALKey, WALEdit> logEntry, Durability durability) throws IOException {
1845    if (entry == null) {
1846      // return an empty array
1847      return Collections.emptyList();
1848    }
1849
1850    long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
1851      entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
1852    int count = entry.getAssociatedCellCount();
1853    List<MutationReplay> mutations = new ArrayList<>();
1854    Cell previousCell = null;
1855    Mutation m = null;
1856    WALKeyImpl key = null;
1857    WALEdit val = null;
1858    if (logEntry != null) {
1859      val = new WALEdit();
1860    }
1861
1862    for (int i = 0; i < count; i++) {
1863      // Throw index out of bounds if our cell count is off
1864      if (!cells.advance()) {
1865        throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
1866      }
1867      Cell cell = cells.current();
1868      if (val != null) val.add(cell);
1869
1870      boolean isNewRowOrType =
1871          previousCell == null || previousCell.getTypeByte() != cell.getTypeByte()
1872              || !CellUtil.matchingRows(previousCell, cell);
1873      if (isNewRowOrType) {
1874        // Create new mutation
1875        if (CellUtil.isDelete(cell)) {
1876          m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1877          // Deletes don't have nonces.
1878          mutations.add(new MutationReplay(
1879              MutationType.DELETE, m, HConstants.NO_NONCE, HConstants.NO_NONCE));
1880        } else {
1881          m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1882          // Puts might come from increment or append, thus we need nonces.
1883          long nonceGroup = entry.getKey().hasNonceGroup()
1884              ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
1885          long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
1886          mutations.add(new MutationReplay(MutationType.PUT, m, nonceGroup, nonce));
1887        }
1888      }
1889      if (CellUtil.isDelete(cell)) {
1890        ((Delete) m).add(cell);
1891      } else {
1892        ((Put) m).add(cell);
1893      }
1894      m.setDurability(durability);
1895      previousCell = cell;
1896    }
1897
1898    // reconstruct WALKey
1899    if (logEntry != null) {
1900      org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey walKeyProto =
1901          entry.getKey();
1902      List<UUID> clusterIds = new ArrayList<>(walKeyProto.getClusterIdsCount());
1903      for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
1904        clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
1905      }
1906      key = new WALKeyImpl(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
1907              walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(),
1908              clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce(), null);
1909      logEntry.setFirst(key);
1910      logEntry.setSecond(val);
1911    }
1912
1913    return mutations;
1914  }
1915}