1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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.HashSet;
29 import java.util.LinkedList;
30 import java.util.List;
31 import java.util.Map;
32 import java.util.NavigableSet;
33 import java.util.Set;
34 import java.util.TreeMap;
35 import java.util.TreeSet;
36 import java.util.UUID;
37 import java.util.concurrent.Callable;
38 import java.util.concurrent.CompletionService;
39 import java.util.concurrent.ConcurrentHashMap;
40 import java.util.concurrent.ExecutionException;
41 import java.util.concurrent.ExecutorCompletionService;
42 import java.util.concurrent.Future;
43 import java.util.concurrent.ThreadFactory;
44 import java.util.concurrent.ThreadPoolExecutor;
45 import java.util.concurrent.TimeUnit;
46 import java.util.concurrent.atomic.AtomicBoolean;
47 import java.util.concurrent.atomic.AtomicLong;
48 import java.util.concurrent.atomic.AtomicReference;
49 import java.util.regex.Matcher;
50 import java.util.regex.Pattern;
51
52 import org.apache.commons.logging.Log;
53 import org.apache.commons.logging.LogFactory;
54 import org.apache.hadoop.conf.Configuration;
55 import org.apache.hadoop.fs.FileAlreadyExistsException;
56 import org.apache.hadoop.fs.FileStatus;
57 import org.apache.hadoop.fs.FileSystem;
58 import org.apache.hadoop.fs.Path;
59 import org.apache.hadoop.fs.PathFilter;
60 import org.apache.hadoop.hbase.Cell;
61 import org.apache.hadoop.hbase.CellScanner;
62 import org.apache.hadoop.hbase.CellUtil;
63 import org.apache.hadoop.hbase.CoordinatedStateException;
64 import org.apache.hadoop.hbase.CoordinatedStateManager;
65 import org.apache.hadoop.hbase.HBaseConfiguration;
66 import org.apache.hadoop.hbase.HConstants;
67 import org.apache.hadoop.hbase.HRegionInfo;
68 import org.apache.hadoop.hbase.HRegionLocation;
69 import org.apache.hadoop.hbase.RemoteExceptionHandler;
70 import org.apache.hadoop.hbase.ServerName;
71 import org.apache.hadoop.hbase.TableName;
72 import org.apache.hadoop.hbase.TableNotFoundException;
73 import org.apache.hadoop.hbase.TableStateManager;
74 import org.apache.hadoop.hbase.Tag;
75 import org.apache.hadoop.hbase.TagRewriteCell;
76 import org.apache.hadoop.hbase.TagType;
77 import org.apache.hadoop.hbase.classification.InterfaceAudience;
78 import org.apache.hadoop.hbase.client.ConnectionUtils;
79 import org.apache.hadoop.hbase.client.Delete;
80 import org.apache.hadoop.hbase.client.Durability;
81 import org.apache.hadoop.hbase.client.HConnection;
82 import org.apache.hadoop.hbase.client.HConnectionManager;
83 import org.apache.hadoop.hbase.client.Mutation;
84 import org.apache.hadoop.hbase.client.Put;
85 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
86 import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
87 import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
88 import org.apache.hadoop.hbase.io.HeapSize;
89 import org.apache.hadoop.hbase.master.SplitLogManager;
90 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
91 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
92 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
93 import org.apache.hadoop.hbase.protobuf.RequestConverter;
94 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
95 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
96 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
97 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
98 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
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.RegionStoreSequenceIds;
103 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
104 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
105 import org.apache.hadoop.hbase.regionserver.HRegion;
106 import org.apache.hadoop.hbase.regionserver.LastSequenceId;
107
108 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
109 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
110 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
111 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
112 import org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink;
113 import org.apache.hadoop.hbase.util.Bytes;
114 import org.apache.hadoop.hbase.util.CancelableProgressable;
115 import org.apache.hadoop.hbase.util.ClassSize;
116 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
117 import org.apache.hadoop.hbase.util.FSUtils;
118 import org.apache.hadoop.hbase.util.Pair;
119 import org.apache.hadoop.hbase.util.Threads;
120 import org.apache.hadoop.hbase.wal.WAL.Entry;
121 import org.apache.hadoop.hbase.wal.WAL.Reader;
122 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
123 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
124 import org.apache.hadoop.io.MultipleIOException;
125
126 import com.google.common.base.Preconditions;
127 import com.google.common.collect.Lists;
128 import com.google.protobuf.ServiceException;
129
130 import com.google.common.annotations.VisibleForTesting;
131 import com.google.common.base.Preconditions;
132 import com.google.common.collect.Lists;
133 import com.google.protobuf.ServiceException;
134
135
136
137
138
139
140 @InterfaceAudience.Private
141 public class WALSplitter {
142 static final Log LOG = LogFactory.getLog(WALSplitter.class);
143
144
145 public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
146
147
148 protected final Path rootDir;
149 protected final FileSystem fs;
150 protected final Configuration conf;
151
152
153
154 OutputSink outputSink;
155 EntryBuffers entryBuffers;
156
157 private Set<TableName> disablingOrDisabledTables =
158 new HashSet<TableName>();
159 private BaseCoordinatedStateManager csm;
160 private final WALFactory walFactory;
161
162
163
164 protected AtomicReference<Throwable> thrown = new AtomicReference<Throwable>();
165
166
167
168 final Object dataAvailable = new Object();
169
170 private MonitoredTask status;
171
172
173 protected final LastSequenceId sequenceIdChecker;
174
175 protected boolean distributedLogReplay;
176
177
178 protected Map<String, Long> lastFlushedSequenceIds = new ConcurrentHashMap<String, Long>();
179
180
181 protected Map<String, Map<byte[], Long>> regionMaxSeqIdInStores =
182 new ConcurrentHashMap<String, Map<byte[], Long>>();
183
184
185 protected String failedServerName = "";
186
187
188 private final int numWriterThreads;
189
190
191 private final int minBatchSize;
192
193 WALSplitter(final WALFactory factory, Configuration conf, Path rootDir,
194 FileSystem fs, LastSequenceId idChecker,
195 CoordinatedStateManager csm, RecoveryMode mode) {
196 this.conf = HBaseConfiguration.create(conf);
197 String codecClassName = conf
198 .get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
199 this.conf.set(HConstants.RPC_CODEC_CONF_KEY, codecClassName);
200 this.rootDir = rootDir;
201 this.fs = fs;
202 this.sequenceIdChecker = idChecker;
203 this.csm = (BaseCoordinatedStateManager)csm;
204 this.walFactory = factory;
205
206 entryBuffers = new EntryBuffers(
207 this.conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
208 128*1024*1024));
209
210
211
212 this.minBatchSize = this.conf.getInt("hbase.regionserver.wal.logreplay.batch.size", 64);
213 this.distributedLogReplay = (RecoveryMode.LOG_REPLAY == mode);
214
215 this.numWriterThreads = this.conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
216 if (csm != null && this.distributedLogReplay) {
217 outputSink = new LogReplayOutputSink(numWriterThreads);
218 } else {
219 if (this.distributedLogReplay) {
220 LOG.info("ZooKeeperWatcher is passed in as NULL so disable distrubitedLogRepaly.");
221 }
222 this.distributedLogReplay = false;
223 outputSink = new LogRecoveredEditsOutputSink(numWriterThreads);
224 }
225
226 }
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244 public static boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs,
245 Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
246 CoordinatedStateManager cp, RecoveryMode mode, final WALFactory factory) throws IOException {
247 WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, idChecker, cp, mode);
248 return s.splitLogFile(logfile, reporter);
249 }
250
251
252
253
254
255 public static List<Path> split(Path rootDir, Path logDir, Path oldLogDir,
256 FileSystem fs, Configuration conf, final WALFactory factory) throws IOException {
257 final FileStatus[] logfiles = SplitLogManager.getFileList(conf,
258 Collections.singletonList(logDir), null);
259 List<Path> splits = new ArrayList<Path>();
260 if (logfiles != null && logfiles.length > 0) {
261 for (FileStatus logfile: logfiles) {
262 WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, null, null,
263 RecoveryMode.LOG_SPLITTING);
264 if (s.splitLogFile(logfile, null)) {
265 finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
266 if (s.outputSink.splits != null) {
267 splits.addAll(s.outputSink.splits);
268 }
269 }
270 }
271 }
272 if (!fs.delete(logDir, true)) {
273 throw new IOException("Unable to delete src dir: " + logDir);
274 }
275 return splits;
276 }
277
278
279
280
281
282 boolean splitLogFile(FileStatus logfile, CancelableProgressable reporter) throws IOException {
283 Preconditions.checkState(status == null);
284 Preconditions.checkArgument(logfile.isFile(),
285 "passed in file status is for something other than a regular file.");
286 boolean isCorrupted = false;
287 boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
288 SPLIT_SKIP_ERRORS_DEFAULT);
289 int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
290 Path logPath = logfile.getPath();
291 boolean outputSinkStarted = false;
292 boolean progress_failed = false;
293 int editsCount = 0;
294 int editsSkipped = 0;
295
296 status =
297 TaskMonitor.get().createStatus(
298 "Splitting log file " + logfile.getPath() + "into a temporary staging area.");
299 Reader in = null;
300 try {
301 long logLength = logfile.getLen();
302 LOG.info("Splitting wal: " + logPath + ", length=" + logLength);
303 LOG.info("DistributedLogReplay = " + this.distributedLogReplay);
304 status.setStatus("Opening log file");
305 if (reporter != null && !reporter.progress()) {
306 progress_failed = true;
307 return false;
308 }
309 try {
310 in = getReader(logfile, skipErrors, reporter);
311 } catch (CorruptedLogFileException e) {
312 LOG.warn("Could not get reader, corrupted log file " + logPath, e);
313 ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
314 isCorrupted = true;
315 }
316 if (in == null) {
317 LOG.warn("Nothing to split in log file " + logPath);
318 return true;
319 }
320 if (csm != null) {
321 try {
322 TableStateManager tsm = csm.getTableStateManager();
323 disablingOrDisabledTables = tsm.getTablesInStates(
324 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
325 } catch (CoordinatedStateException e) {
326 throw new IOException("Can't get disabling/disabled tables", e);
327 }
328 }
329 int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
330 int numOpenedFilesLastCheck = 0;
331 outputSink.setReporter(reporter);
332 outputSink.startWriterThreads();
333 outputSinkStarted = true;
334 Entry entry;
335 Long lastFlushedSequenceId = -1L;
336 ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(logPath);
337 failedServerName = (serverName == null) ? "" : serverName.getServerName();
338 while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) {
339 byte[] region = entry.getKey().getEncodedRegionName();
340 String key = Bytes.toString(region);
341 lastFlushedSequenceId = lastFlushedSequenceIds.get(key);
342 if (lastFlushedSequenceId == null) {
343 if (this.distributedLogReplay) {
344 RegionStoreSequenceIds ids =
345 csm.getSplitLogWorkerCoordination().getRegionFlushedSequenceId(failedServerName,
346 key);
347 if (ids != null) {
348 lastFlushedSequenceId = ids.getLastFlushedSequenceId();
349 }
350 } else if (sequenceIdChecker != null) {
351 lastFlushedSequenceId = sequenceIdChecker.getLastSequenceId(region);
352 }
353 if (lastFlushedSequenceId == null) {
354 lastFlushedSequenceId = -1L;
355 }
356 lastFlushedSequenceIds.put(key, lastFlushedSequenceId);
357 }
358 if (lastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
359 editsSkipped++;
360 continue;
361 }
362 entryBuffers.appendEntry(entry);
363 editsCount++;
364 int moreWritersFromLastCheck = this.getNumOpenWriters() - numOpenedFilesLastCheck;
365
366 if (editsCount % interval == 0
367 || moreWritersFromLastCheck > numOpenedFilesBeforeReporting) {
368 numOpenedFilesLastCheck = this.getNumOpenWriters();
369 String countsStr = (editsCount - (editsSkipped + outputSink.getSkippedEdits()))
370 + " edits, skipped " + editsSkipped + " edits.";
371 status.setStatus("Split " + countsStr);
372 if (reporter != null && !reporter.progress()) {
373 progress_failed = true;
374 return false;
375 }
376 }
377 }
378 } catch (InterruptedException ie) {
379 IOException iie = new InterruptedIOException();
380 iie.initCause(ie);
381 throw iie;
382 } catch (CorruptedLogFileException e) {
383 LOG.warn("Could not parse, corrupted log file " + logPath, e);
384 csm.getSplitLogWorkerCoordination().markCorrupted(rootDir,
385 logfile.getPath().getName(), fs);
386 isCorrupted = true;
387 } catch (IOException e) {
388 e = RemoteExceptionHandler.checkIOException(e);
389 throw e;
390 } finally {
391 LOG.debug("Finishing writing output logs and closing down.");
392 try {
393 if (null != in) {
394 in.close();
395 }
396 } catch (IOException exception) {
397 LOG.warn("Could not close wal reader: " + exception.getMessage());
398 LOG.debug("exception details", exception);
399 }
400 try {
401 if (outputSinkStarted) {
402
403
404 progress_failed = true;
405 progress_failed = outputSink.finishWritingAndClose() == null;
406 }
407 } finally {
408 String msg =
409 "Processed " + editsCount + " edits across " + outputSink.getNumberOfRecoveredRegions()
410 + " regions; edits skipped=" + editsSkipped + "; log file=" + logPath +
411 ", length=" + logfile.getLen() +
412 ", corrupted=" + isCorrupted + ", progress failed=" + progress_failed;
413 LOG.info(msg);
414 status.markComplete(msg);
415 }
416 }
417 return !progress_failed;
418 }
419
420
421
422
423
424
425
426
427
428
429
430
431 public static void finishSplitLogFile(String logfile,
432 Configuration conf) throws IOException {
433 Path rootdir = FSUtils.getRootDir(conf);
434 Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
435 Path logPath;
436 if (FSUtils.isStartingWithPath(rootdir, logfile)) {
437 logPath = new Path(logfile);
438 } else {
439 logPath = new Path(rootdir, logfile);
440 }
441 finishSplitLogFile(rootdir, oldLogDir, logPath, conf);
442 }
443
444 static void finishSplitLogFile(Path rootdir, Path oldLogDir,
445 Path logPath, Configuration conf) throws IOException {
446 List<Path> processedLogs = new ArrayList<Path>();
447 List<Path> corruptedLogs = new ArrayList<Path>();
448 FileSystem fs;
449 fs = rootdir.getFileSystem(conf);
450 if (ZKSplitLog.isCorrupted(rootdir, logPath.getName(), fs)) {
451 corruptedLogs.add(logPath);
452 } else {
453 processedLogs.add(logPath);
454 }
455 archiveLogs(corruptedLogs, processedLogs, oldLogDir, fs, conf);
456 Path stagingDir = ZKSplitLog.getSplitLogDir(rootdir, logPath.getName());
457 fs.delete(stagingDir, true);
458 }
459
460
461
462
463
464
465
466
467
468
469
470
471
472 private static void archiveLogs(
473 final List<Path> corruptedLogs,
474 final List<Path> processedLogs, final Path oldLogDir,
475 final FileSystem fs, final Configuration conf) throws IOException {
476 final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get(
477 "hbase.regionserver.hlog.splitlog.corrupt.dir", HConstants.CORRUPT_DIR_NAME));
478
479 if (!fs.mkdirs(corruptDir)) {
480 LOG.info("Unable to mkdir " + corruptDir);
481 }
482 fs.mkdirs(oldLogDir);
483
484
485
486 for (Path corrupted : corruptedLogs) {
487 Path p = new Path(corruptDir, corrupted.getName());
488 if (fs.exists(corrupted)) {
489 if (!fs.rename(corrupted, p)) {
490 LOG.warn("Unable to move corrupted log " + corrupted + " to " + p);
491 } else {
492 LOG.warn("Moved corrupted log " + corrupted + " to " + p);
493 }
494 }
495 }
496
497 for (Path p : processedLogs) {
498 Path newPath = FSHLog.getWALArchivePath(oldLogDir, p);
499 if (fs.exists(p)) {
500 if (!FSUtils.renameAndSetModifyTime(fs, p, newPath)) {
501 LOG.warn("Unable to move " + p + " to " + newPath);
502 } else {
503 LOG.info("Archived processed log " + p + " to " + newPath);
504 }
505 }
506 }
507 }
508
509
510
511
512
513
514
515
516
517
518
519
520
521 @SuppressWarnings("deprecation")
522 static Path getRegionSplitEditsPath(final FileSystem fs,
523 final Entry logEntry, final Path rootDir, boolean isCreate)
524 throws IOException {
525 Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
526 String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
527 Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
528 Path dir = getRegionDirRecoveredEditsDir(regiondir);
529
530 if (!fs.exists(regiondir)) {
531 LOG.info("This region's directory doesn't exist: "
532 + regiondir.toString() + ". It is very likely that it was" +
533 " already split so it's safe to discard those edits.");
534 return null;
535 }
536 if (fs.exists(dir) && fs.isFile(dir)) {
537 Path tmp = new Path("/tmp");
538 if (!fs.exists(tmp)) {
539 fs.mkdirs(tmp);
540 }
541 tmp = new Path(tmp,
542 HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
543 LOG.warn("Found existing old file: " + dir + ". It could be some "
544 + "leftover of an old installation. It should be a folder instead. "
545 + "So moving it to " + tmp);
546 if (!fs.rename(dir, tmp)) {
547 LOG.warn("Failed to sideline old file " + dir);
548 }
549 }
550
551 if (isCreate && !fs.exists(dir)) {
552 if (!fs.mkdirs(dir)) LOG.warn("mkdir failed on " + dir);
553 }
554
555
556 String fileName = formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum());
557 fileName = getTmpRecoveredEditsFileName(fileName);
558 return new Path(dir, fileName);
559 }
560
561 static String getTmpRecoveredEditsFileName(String fileName) {
562 return fileName + RECOVERED_LOG_TMPFILE_SUFFIX;
563 }
564
565
566
567
568
569
570
571
572
573 static Path getCompletedRecoveredEditsFilePath(Path srcPath,
574 Long maximumEditLogSeqNum) {
575 String fileName = formatRecoveredEditsFileName(maximumEditLogSeqNum);
576 return new Path(srcPath.getParent(), fileName);
577 }
578
579 static String formatRecoveredEditsFileName(final long seqid) {
580 return String.format("%019d", seqid);
581 }
582
583 private static final Pattern EDITFILES_NAME_PATTERN = Pattern.compile("-?[0-9]+");
584 private static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
585
586
587
588
589
590
591
592 public static Path getRegionDirRecoveredEditsDir(final Path regiondir) {
593 return new Path(regiondir, HConstants.RECOVERED_EDITS_DIR);
594 }
595
596
597
598
599
600
601
602
603
604
605 public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
606 final Path regiondir) throws IOException {
607 NavigableSet<Path> filesSorted = new TreeSet<Path>();
608 Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
609 if (!fs.exists(editsdir))
610 return filesSorted;
611 FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
612 @Override
613 public boolean accept(Path p) {
614 boolean result = false;
615 try {
616
617
618
619
620 Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
621 result = fs.isFile(p) && m.matches();
622
623
624 if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
625 result = false;
626 }
627
628 if (isSequenceIdFile(p)) {
629 result = false;
630 }
631 } catch (IOException e) {
632 LOG.warn("Failed isFile check on " + p);
633 }
634 return result;
635 }
636 });
637 if (files == null) {
638 return filesSorted;
639 }
640 for (FileStatus status : files) {
641 filesSorted.add(status.getPath());
642 }
643 return filesSorted;
644 }
645
646
647
648
649
650
651
652
653
654
655 public static Path moveAsideBadEditsFile(final FileSystem fs, final Path edits)
656 throws IOException {
657 Path moveAsideName = new Path(edits.getParent(), edits.getName() + "."
658 + System.currentTimeMillis());
659 if (!fs.rename(edits, moveAsideName)) {
660 LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
661 }
662 return moveAsideName;
663 }
664
665 private static final String SEQUENCE_ID_FILE_SUFFIX = ".seqid";
666 private static final String OLD_SEQUENCE_ID_FILE_SUFFIX = "_seqid";
667 private static final int SEQUENCE_ID_FILE_SUFFIX_LENGTH = SEQUENCE_ID_FILE_SUFFIX.length();
668
669
670
671
672 @VisibleForTesting
673 public static boolean isSequenceIdFile(final Path file) {
674 return file.getName().endsWith(SEQUENCE_ID_FILE_SUFFIX)
675 || file.getName().endsWith(OLD_SEQUENCE_ID_FILE_SUFFIX);
676 }
677
678
679
680
681
682
683
684
685
686
687 public static long writeRegionSequenceIdFile(final FileSystem fs, final Path regiondir,
688 long newSeqId, long saftyBumper) throws IOException {
689
690 Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
691 long maxSeqId = 0;
692 FileStatus[] files = null;
693 if (fs.exists(editsdir)) {
694 files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
695 @Override
696 public boolean accept(Path p) {
697 return isSequenceIdFile(p);
698 }
699 });
700 if (files != null) {
701 for (FileStatus status : files) {
702 String fileName = status.getPath().getName();
703 try {
704 Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length()
705 - SEQUENCE_ID_FILE_SUFFIX_LENGTH));
706 maxSeqId = Math.max(tmpSeqId, maxSeqId);
707 } catch (NumberFormatException ex) {
708 LOG.warn("Invalid SeqId File Name=" + fileName);
709 }
710 }
711 }
712 }
713 if (maxSeqId > newSeqId) {
714 newSeqId = maxSeqId;
715 }
716 newSeqId += saftyBumper;
717
718
719 Path newSeqIdFile = new Path(editsdir, newSeqId + SEQUENCE_ID_FILE_SUFFIX);
720 if (newSeqId != maxSeqId) {
721 try {
722 if (!fs.createNewFile(newSeqIdFile) && !fs.exists(newSeqIdFile)) {
723 throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
724 }
725 if (LOG.isDebugEnabled()) {
726 LOG.debug("Wrote region seqId=" + newSeqIdFile + " to file, newSeqId=" + newSeqId
727 + ", maxSeqId=" + maxSeqId);
728 }
729 } catch (FileAlreadyExistsException ignored) {
730
731 }
732 }
733
734 if (files != null) {
735 for (FileStatus status : files) {
736 if (newSeqIdFile.equals(status.getPath())) {
737 continue;
738 }
739 fs.delete(status.getPath(), false);
740 }
741 }
742 return newSeqId;
743 }
744
745
746
747
748
749
750
751
752
753 protected Reader getReader(FileStatus file, boolean skipErrors, CancelableProgressable reporter)
754 throws IOException, CorruptedLogFileException {
755 Path path = file.getPath();
756 long length = file.getLen();
757 Reader in;
758
759
760
761
762 if (length <= 0) {
763 LOG.warn("File " + path + " might be still open, length is 0");
764 }
765
766 try {
767 FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf, reporter);
768 try {
769 in = getReader(path, reporter);
770 } catch (EOFException e) {
771 if (length <= 0) {
772
773
774
775
776
777 LOG.warn("Could not open " + path + " for reading. File is empty", e);
778 return null;
779 } else {
780
781 return null;
782 }
783 }
784 } catch (IOException e) {
785 if (e instanceof FileNotFoundException) {
786
787 LOG.warn("File " + path + " doesn't exist anymore.", e);
788 return null;
789 }
790 if (!skipErrors || e instanceof InterruptedIOException) {
791 throw e;
792 }
793 CorruptedLogFileException t =
794 new CorruptedLogFileException("skipErrors=true Could not open wal " +
795 path + " ignoring");
796 t.initCause(e);
797 throw t;
798 }
799 return in;
800 }
801
802 static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
803 throws CorruptedLogFileException, IOException {
804 try {
805 return in.next();
806 } catch (EOFException eof) {
807
808 LOG.info("EOF from wal " + path + ". continuing");
809 return null;
810 } catch (IOException e) {
811
812
813 if (e.getCause() != null &&
814 (e.getCause() instanceof ParseException ||
815 e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
816 LOG.warn("Parse exception " + e.getCause().toString() + " from wal "
817 + path + ". continuing");
818 return null;
819 }
820 if (!skipErrors) {
821 throw e;
822 }
823 CorruptedLogFileException t =
824 new CorruptedLogFileException("skipErrors=true Ignoring exception" +
825 " while parsing wal " + path + ". Marking as corrupted");
826 t.initCause(e);
827 throw t;
828 }
829 }
830
831 private void writerThreadError(Throwable t) {
832 thrown.compareAndSet(null, t);
833 }
834
835
836
837
838 private void checkForErrors() throws IOException {
839 Throwable thrown = this.thrown.get();
840 if (thrown == null) return;
841 if (thrown instanceof IOException) {
842 throw new IOException(thrown);
843 } else {
844 throw new RuntimeException(thrown);
845 }
846 }
847
848
849
850
851 protected Writer createWriter(Path logfile)
852 throws IOException {
853 return walFactory.createRecoveredEditsWriter(fs, logfile);
854 }
855
856
857
858
859
860 protected Reader getReader(Path curLogFile, CancelableProgressable reporter) throws IOException {
861 return walFactory.createReader(fs, curLogFile, reporter);
862 }
863
864
865
866
867 private int getNumOpenWriters() {
868 int result = 0;
869 if (this.outputSink != null) {
870 result += this.outputSink.getNumOpenWriters();
871 }
872 return result;
873 }
874
875
876
877
878
879
880
881
882 class EntryBuffers {
883 Map<byte[], RegionEntryBuffer> buffers =
884 new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
885
886
887
888
889 Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
890
891 long totalBuffered = 0;
892 long maxHeapUsage;
893
894 EntryBuffers(long maxHeapUsage) {
895 this.maxHeapUsage = maxHeapUsage;
896 }
897
898
899
900
901
902
903
904
905 void appendEntry(Entry entry) throws InterruptedException, IOException {
906 WALKey key = entry.getKey();
907
908 RegionEntryBuffer buffer;
909 long incrHeap;
910 synchronized (this) {
911 buffer = buffers.get(key.getEncodedRegionName());
912 if (buffer == null) {
913 buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
914 buffers.put(key.getEncodedRegionName(), buffer);
915 }
916 incrHeap= buffer.appendEntry(entry);
917 }
918
919
920 synchronized (dataAvailable) {
921 totalBuffered += incrHeap;
922 while (totalBuffered > maxHeapUsage && thrown.get() == null) {
923 LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
924 dataAvailable.wait(2000);
925 }
926 dataAvailable.notifyAll();
927 }
928 checkForErrors();
929 }
930
931
932
933
934 synchronized RegionEntryBuffer getChunkToWrite() {
935 long biggestSize = 0;
936 byte[] biggestBufferKey = null;
937
938 for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
939 long size = entry.getValue().heapSize();
940 if (size > biggestSize && (!currentlyWriting.contains(entry.getKey()))) {
941 biggestSize = size;
942 biggestBufferKey = entry.getKey();
943 }
944 }
945 if (biggestBufferKey == null) {
946 return null;
947 }
948
949 RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
950 currentlyWriting.add(biggestBufferKey);
951 return buffer;
952 }
953
954 void doneWriting(RegionEntryBuffer buffer) {
955 synchronized (this) {
956 boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
957 assert removed;
958 }
959 long size = buffer.heapSize();
960
961 synchronized (dataAvailable) {
962 totalBuffered -= size;
963
964 dataAvailable.notifyAll();
965 }
966 }
967
968 synchronized boolean isRegionCurrentlyWriting(byte[] region) {
969 return currentlyWriting.contains(region);
970 }
971 }
972
973
974
975
976
977
978
979 static class RegionEntryBuffer implements HeapSize {
980 long heapInBuffer = 0;
981 List<Entry> entryBuffer;
982 TableName tableName;
983 byte[] encodedRegionName;
984
985 RegionEntryBuffer(TableName tableName, byte[] region) {
986 this.tableName = tableName;
987 this.encodedRegionName = region;
988 this.entryBuffer = new LinkedList<Entry>();
989 }
990
991 long appendEntry(Entry entry) {
992 internify(entry);
993 entryBuffer.add(entry);
994 long incrHeap = entry.getEdit().heapSize() +
995 ClassSize.align(2 * ClassSize.REFERENCE) +
996 0;
997 heapInBuffer += incrHeap;
998 return incrHeap;
999 }
1000
1001 private void internify(Entry entry) {
1002 WALKey k = entry.getKey();
1003 k.internTableName(this.tableName);
1004 k.internEncodedRegionName(this.encodedRegionName);
1005 }
1006
1007 @Override
1008 public long heapSize() {
1009 return heapInBuffer;
1010 }
1011 }
1012
1013 class WriterThread extends Thread {
1014 private volatile boolean shouldStop = false;
1015 private OutputSink outputSink = null;
1016
1017 WriterThread(OutputSink sink, int i) {
1018 super(Thread.currentThread().getName() + "-Writer-" + i);
1019 outputSink = sink;
1020 }
1021
1022 @Override
1023 public void run() {
1024 try {
1025 doRun();
1026 } catch (Throwable t) {
1027 LOG.error("Exiting thread", t);
1028 writerThreadError(t);
1029 }
1030 }
1031
1032 private void doRun() throws IOException {
1033 LOG.debug("Writer thread " + this + ": starting");
1034 while (true) {
1035 RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
1036 if (buffer == null) {
1037
1038 synchronized (dataAvailable) {
1039 if (shouldStop && !this.outputSink.flush()) {
1040 return;
1041 }
1042 try {
1043 dataAvailable.wait(500);
1044 } catch (InterruptedException ie) {
1045 if (!shouldStop) {
1046 throw new RuntimeException(ie);
1047 }
1048 }
1049 }
1050 continue;
1051 }
1052
1053 assert buffer != null;
1054 try {
1055 writeBuffer(buffer);
1056 } finally {
1057 entryBuffers.doneWriting(buffer);
1058 }
1059 }
1060 }
1061
1062 private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
1063 outputSink.append(buffer);
1064 }
1065
1066 void finish() {
1067 synchronized (dataAvailable) {
1068 shouldStop = true;
1069 dataAvailable.notifyAll();
1070 }
1071 }
1072 }
1073
1074
1075
1076
1077
1078 abstract class OutputSink {
1079
1080 protected Map<byte[], SinkWriter> writers = Collections
1081 .synchronizedMap(new TreeMap<byte[], SinkWriter>(Bytes.BYTES_COMPARATOR));;
1082
1083 protected final Map<byte[], Long> regionMaximumEditLogSeqNum = Collections
1084 .synchronizedMap(new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR));
1085
1086 protected final List<WriterThread> writerThreads = Lists.newArrayList();
1087
1088
1089 protected final Set<byte[]> blacklistedRegions = Collections
1090 .synchronizedSet(new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR));
1091
1092 protected boolean closeAndCleanCompleted = false;
1093
1094 protected boolean writersClosed = false;
1095
1096 protected final int numThreads;
1097
1098 protected CancelableProgressable reporter = null;
1099
1100 protected AtomicLong skippedEdits = new AtomicLong();
1101
1102 protected List<Path> splits = null;
1103
1104 public OutputSink(int numWriters) {
1105 numThreads = numWriters;
1106 }
1107
1108 void setReporter(CancelableProgressable reporter) {
1109 this.reporter = reporter;
1110 }
1111
1112
1113
1114
1115 synchronized void startWriterThreads() {
1116 for (int i = 0; i < numThreads; i++) {
1117 WriterThread t = new WriterThread(this, i);
1118 t.start();
1119 writerThreads.add(t);
1120 }
1121 }
1122
1123
1124
1125
1126
1127 void updateRegionMaximumEditLogSeqNum(Entry entry) {
1128 synchronized (regionMaximumEditLogSeqNum) {
1129 Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(entry.getKey()
1130 .getEncodedRegionName());
1131 if (currentMaxSeqNum == null || entry.getKey().getLogSeqNum() > currentMaxSeqNum) {
1132 regionMaximumEditLogSeqNum.put(entry.getKey().getEncodedRegionName(), entry.getKey()
1133 .getLogSeqNum());
1134 }
1135 }
1136 }
1137
1138 Long getRegionMaximumEditLogSeqNum(byte[] region) {
1139 return regionMaximumEditLogSeqNum.get(region);
1140 }
1141
1142
1143
1144
1145 int getNumOpenWriters() {
1146 return this.writers.size();
1147 }
1148
1149 long getSkippedEdits() {
1150 return this.skippedEdits.get();
1151 }
1152
1153
1154
1155
1156
1157
1158 protected boolean finishWriting() throws IOException {
1159 LOG.debug("Waiting for split writer threads to finish");
1160 boolean progress_failed = false;
1161 for (WriterThread t : writerThreads) {
1162 t.finish();
1163 }
1164 for (WriterThread t : writerThreads) {
1165 if (!progress_failed && reporter != null && !reporter.progress()) {
1166 progress_failed = true;
1167 }
1168 try {
1169 t.join();
1170 } catch (InterruptedException ie) {
1171 IOException iie = new InterruptedIOException();
1172 iie.initCause(ie);
1173 throw iie;
1174 }
1175 }
1176 checkForErrors();
1177 LOG.info("Split writers finished");
1178 return (!progress_failed);
1179 }
1180
1181 abstract List<Path> finishWritingAndClose() throws IOException;
1182
1183
1184
1185
1186 abstract Map<byte[], Long> getOutputCounts();
1187
1188
1189
1190
1191 abstract int getNumberOfRecoveredRegions();
1192
1193
1194
1195
1196
1197 abstract void append(RegionEntryBuffer buffer) throws IOException;
1198
1199
1200
1201
1202
1203 protected boolean flush() throws IOException {
1204 return false;
1205 }
1206 }
1207
1208
1209
1210
1211 class LogRecoveredEditsOutputSink extends OutputSink {
1212
1213 public LogRecoveredEditsOutputSink(int numWriters) {
1214
1215
1216
1217
1218
1219 super(numWriters);
1220 }
1221
1222
1223
1224
1225
1226 @Override
1227 List<Path> finishWritingAndClose() throws IOException {
1228 boolean isSuccessful = false;
1229 List<Path> result = null;
1230 try {
1231 isSuccessful = finishWriting();
1232 } finally {
1233 result = close();
1234 List<IOException> thrown = closeLogWriters(null);
1235 if (thrown != null && !thrown.isEmpty()) {
1236 throw MultipleIOException.createIOException(thrown);
1237 }
1238 }
1239 if (isSuccessful) {
1240 splits = result;
1241 }
1242 return splits;
1243 }
1244
1245
1246
1247
1248
1249 private List<Path> close() throws IOException {
1250 Preconditions.checkState(!closeAndCleanCompleted);
1251
1252 final List<Path> paths = new ArrayList<Path>();
1253 final List<IOException> thrown = Lists.newArrayList();
1254 ThreadPoolExecutor closeThreadPool = Threads.getBoundedCachedThreadPool(numThreads, 30L,
1255 TimeUnit.SECONDS, new ThreadFactory() {
1256 private int count = 1;
1257
1258 @Override
1259 public Thread newThread(Runnable r) {
1260 Thread t = new Thread(r, "split-log-closeStream-" + count++);
1261 return t;
1262 }
1263 });
1264 CompletionService<Void> completionService =
1265 new ExecutorCompletionService<Void>(closeThreadPool);
1266 for (final Map.Entry<byte[], SinkWriter> writersEntry : writers.entrySet()) {
1267 LOG.debug("Submitting close of " + ((WriterAndPath)writersEntry.getValue()).p);
1268 completionService.submit(new Callable<Void>() {
1269 @Override
1270 public Void call() throws Exception {
1271 WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
1272 LOG.debug("Closing " + wap.p);
1273 try {
1274 wap.w.close();
1275 } catch (IOException ioe) {
1276 LOG.error("Couldn't close log at " + wap.p, ioe);
1277 thrown.add(ioe);
1278 return null;
1279 }
1280 LOG.info("Closed wap " + wap.p + " (wrote " + wap.editsWritten + " edits in "
1281 + (wap.nanosSpent / 1000 / 1000) + "ms)");
1282
1283 if (wap.editsWritten == 0) {
1284
1285 if (fs.exists(wap.p) && !fs.delete(wap.p, false)) {
1286 LOG.warn("Failed deleting empty " + wap.p);
1287 throw new IOException("Failed deleting empty " + wap.p);
1288 }
1289 return null;
1290 }
1291
1292 Path dst = getCompletedRecoveredEditsFilePath(wap.p,
1293 regionMaximumEditLogSeqNum.get(writersEntry.getKey()));
1294 try {
1295 if (!dst.equals(wap.p) && fs.exists(dst)) {
1296 LOG.warn("Found existing old edits file. It could be the "
1297 + "result of a previous failed split attempt. Deleting " + dst + ", length="
1298 + fs.getFileStatus(dst).getLen());
1299 if (!fs.delete(dst, false)) {
1300 LOG.warn("Failed deleting of old " + dst);
1301 throw new IOException("Failed deleting of old " + dst);
1302 }
1303 }
1304
1305
1306
1307 if (fs.exists(wap.p)) {
1308 if (!fs.rename(wap.p, dst)) {
1309 throw new IOException("Failed renaming " + wap.p + " to " + dst);
1310 }
1311 LOG.info("Rename " + wap.p + " to " + dst);
1312 }
1313 } catch (IOException ioe) {
1314 LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
1315 thrown.add(ioe);
1316 return null;
1317 }
1318 paths.add(dst);
1319 return null;
1320 }
1321 });
1322 }
1323
1324 boolean progress_failed = false;
1325 try {
1326 for (int i = 0, n = this.writers.size(); i < n; i++) {
1327 Future<Void> future = completionService.take();
1328 future.get();
1329 if (!progress_failed && reporter != null && !reporter.progress()) {
1330 progress_failed = true;
1331 }
1332 }
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
1343 if (!thrown.isEmpty()) {
1344 throw MultipleIOException.createIOException(thrown);
1345 }
1346 writersClosed = true;
1347 closeAndCleanCompleted = true;
1348 if (progress_failed) {
1349 return null;
1350 }
1351 return paths;
1352 }
1353
1354 private List<IOException> closeLogWriters(List<IOException> thrown) throws IOException {
1355 if (writersClosed) {
1356 return thrown;
1357 }
1358
1359 if (thrown == null) {
1360 thrown = Lists.newArrayList();
1361 }
1362 try {
1363 for (WriterThread t : writerThreads) {
1364 while (t.isAlive()) {
1365 t.shouldStop = true;
1366 t.interrupt();
1367 try {
1368 t.join(10);
1369 } catch (InterruptedException e) {
1370 IOException iie = new InterruptedIOException();
1371 iie.initCause(e);
1372 throw iie;
1373 }
1374 }
1375 }
1376 } finally {
1377 synchronized (writers) {
1378 WriterAndPath wap = null;
1379 for (SinkWriter tmpWAP : writers.values()) {
1380 try {
1381 wap = (WriterAndPath) tmpWAP;
1382 wap.w.close();
1383 } catch (IOException ioe) {
1384 LOG.error("Couldn't close log at " + wap.p, ioe);
1385 thrown.add(ioe);
1386 continue;
1387 }
1388 LOG.info("Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in "
1389 + (wap.nanosSpent / 1000 / 1000) + "ms)");
1390 }
1391 }
1392 writersClosed = true;
1393 }
1394
1395 return thrown;
1396 }
1397
1398
1399
1400
1401
1402
1403 private WriterAndPath getWriterAndPath(Entry entry) throws IOException {
1404 byte region[] = entry.getKey().getEncodedRegionName();
1405 WriterAndPath ret = (WriterAndPath) writers.get(region);
1406 if (ret != null) {
1407 return ret;
1408 }
1409
1410
1411 if (blacklistedRegions.contains(region)) {
1412 return null;
1413 }
1414 ret = createWAP(region, entry, rootDir);
1415 if (ret == null) {
1416 blacklistedRegions.add(region);
1417 return null;
1418 }
1419 writers.put(region, ret);
1420 return ret;
1421 }
1422
1423
1424
1425
1426 private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
1427 Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, true);
1428 if (regionedits == null) {
1429 return null;
1430 }
1431 if (fs.exists(regionedits)) {
1432 LOG.warn("Found old edits file. It could be the "
1433 + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
1434 + fs.getFileStatus(regionedits).getLen());
1435 if (!fs.delete(regionedits, false)) {
1436 LOG.warn("Failed delete of old " + regionedits);
1437 }
1438 }
1439 Writer w = createWriter(regionedits);
1440 LOG.info("Creating writer path=" + regionedits + " region=" + Bytes.toStringBinary(region));
1441 return (new WriterAndPath(regionedits, w));
1442 }
1443
1444 @Override
1445 void append(RegionEntryBuffer buffer) throws IOException {
1446 List<Entry> entries = buffer.entryBuffer;
1447 if (entries.isEmpty()) {
1448 LOG.warn("got an empty buffer, skipping");
1449 return;
1450 }
1451
1452 WriterAndPath wap = null;
1453
1454 long startTime = System.nanoTime();
1455 try {
1456 int editsCount = 0;
1457
1458 for (Entry logEntry : entries) {
1459 if (wap == null) {
1460 wap = getWriterAndPath(logEntry);
1461 if (wap == null) {
1462 if (LOG.isDebugEnabled()) {
1463 LOG.debug("getWriterAndPath decided we don't need to write edits for " + logEntry);
1464 }
1465 return;
1466 }
1467 }
1468 wap.w.append(logEntry);
1469 this.updateRegionMaximumEditLogSeqNum(logEntry);
1470 editsCount++;
1471 }
1472
1473 wap.incrementEdits(editsCount);
1474 wap.incrementNanoTime(System.nanoTime() - startTime);
1475 } catch (IOException e) {
1476 e = RemoteExceptionHandler.checkIOException(e);
1477 LOG.fatal(" Got while writing log entry to log", e);
1478 throw e;
1479 }
1480 }
1481
1482
1483
1484
1485 @Override
1486 Map<byte[], Long> getOutputCounts() {
1487 TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1488 synchronized (writers) {
1489 for (Map.Entry<byte[], SinkWriter> entry : writers.entrySet()) {
1490 ret.put(entry.getKey(), entry.getValue().editsWritten);
1491 }
1492 }
1493 return ret;
1494 }
1495
1496 @Override
1497 int getNumberOfRecoveredRegions() {
1498 return writers.size();
1499 }
1500 }
1501
1502
1503
1504
1505 private abstract static class SinkWriter {
1506
1507 long editsWritten = 0;
1508
1509 long nanosSpent = 0;
1510
1511 void incrementEdits(int edits) {
1512 editsWritten += edits;
1513 }
1514
1515 void incrementNanoTime(long nanos) {
1516 nanosSpent += nanos;
1517 }
1518 }
1519
1520
1521
1522
1523
1524 private final static class WriterAndPath extends SinkWriter {
1525 final Path p;
1526 final Writer w;
1527
1528 WriterAndPath(final Path p, final Writer w) {
1529 this.p = p;
1530 this.w = w;
1531 }
1532 }
1533
1534
1535
1536
1537 class LogReplayOutputSink extends OutputSink {
1538 private static final double BUFFER_THRESHOLD = 0.35;
1539 private static final String KEY_DELIMITER = "#";
1540
1541 private long waitRegionOnlineTimeOut;
1542 private final Set<String> recoveredRegions = Collections.synchronizedSet(new HashSet<String>());
1543 private final Map<String, RegionServerWriter> writers =
1544 new ConcurrentHashMap<String, RegionServerWriter>();
1545
1546 private final Map<String, HRegionLocation> onlineRegions =
1547 new ConcurrentHashMap<String, HRegionLocation>();
1548
1549 private Map<TableName, HConnection> tableNameToHConnectionMap = Collections
1550 .synchronizedMap(new TreeMap<TableName, HConnection>());
1551
1552
1553
1554
1555 private Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
1556 new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();
1557 private List<Throwable> thrown = new ArrayList<Throwable>();
1558
1559
1560
1561
1562
1563 private LogRecoveredEditsOutputSink logRecoveredEditsOutputSink;
1564 private boolean hasEditsInDisablingOrDisabledTables = false;
1565
1566 public LogReplayOutputSink(int numWriters) {
1567 super(numWriters);
1568 this.waitRegionOnlineTimeOut =
1569 conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT,
1570 ZKSplitLogManagerCoordination.DEFAULT_TIMEOUT);
1571 this.logRecoveredEditsOutputSink = new LogRecoveredEditsOutputSink(numWriters);
1572 this.logRecoveredEditsOutputSink.setReporter(reporter);
1573 }
1574
1575 @Override
1576 void append(RegionEntryBuffer buffer) throws IOException {
1577 List<Entry> entries = buffer.entryBuffer;
1578 if (entries.isEmpty()) {
1579 LOG.warn("got an empty buffer, skipping");
1580 return;
1581 }
1582
1583
1584 if (disablingOrDisabledTables.contains(buffer.tableName)) {
1585
1586 logRecoveredEditsOutputSink.append(buffer);
1587 hasEditsInDisablingOrDisabledTables = true;
1588
1589 addToRecoveredRegions(Bytes.toString(buffer.encodedRegionName));
1590 return;
1591 }
1592
1593
1594 groupEditsByServer(entries);
1595
1596
1597 String maxLocKey = null;
1598 int maxSize = 0;
1599 List<Pair<HRegionLocation, Entry>> maxQueue = null;
1600 synchronized (this.serverToBufferQueueMap) {
1601 for (String key : this.serverToBufferQueueMap.keySet()) {
1602 List<Pair<HRegionLocation, Entry>> curQueue = this.serverToBufferQueueMap.get(key);
1603 if (curQueue.size() > maxSize) {
1604 maxSize = curQueue.size();
1605 maxQueue = curQueue;
1606 maxLocKey = key;
1607 }
1608 }
1609 if (maxSize < minBatchSize
1610 && entryBuffers.totalBuffered < BUFFER_THRESHOLD * entryBuffers.maxHeapUsage) {
1611
1612 return;
1613 } else if (maxSize > 0) {
1614 this.serverToBufferQueueMap.remove(maxLocKey);
1615 }
1616 }
1617
1618 if (maxSize > 0) {
1619 processWorkItems(maxLocKey, maxQueue);
1620 }
1621 }
1622
1623 private void addToRecoveredRegions(String encodedRegionName) {
1624 if (!recoveredRegions.contains(encodedRegionName)) {
1625 recoveredRegions.add(encodedRegionName);
1626 }
1627 }
1628
1629
1630
1631
1632
1633 private void groupEditsByServer(List<Entry> entries) throws IOException {
1634 Set<TableName> nonExistentTables = null;
1635 Long cachedLastFlushedSequenceId = -1l;
1636 for (Entry entry : entries) {
1637 WALEdit edit = entry.getEdit();
1638 TableName table = entry.getKey().getTablename();
1639
1640 entry.getKey().setScopes(null);
1641 String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
1642
1643 if (nonExistentTables != null && nonExistentTables.contains(table)) {
1644 this.skippedEdits.incrementAndGet();
1645 continue;
1646 }
1647
1648 Map<byte[], Long> maxStoreSequenceIds = null;
1649 boolean needSkip = false;
1650 HRegionLocation loc = null;
1651 String locKey = null;
1652 List<Cell> cells = edit.getCells();
1653 List<Cell> skippedCells = new ArrayList<Cell>();
1654 HConnection hconn = this.getConnectionByTableName(table);
1655
1656 for (Cell cell : cells) {
1657 byte[] row = cell.getRow();
1658 byte[] family = cell.getFamily();
1659 boolean isCompactionEntry = false;
1660 if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
1661 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
1662 if (compaction != null && compaction.hasRegionName()) {
1663 try {
1664 byte[][] regionName = HRegionInfo.parseRegionName(compaction.getRegionName()
1665 .toByteArray());
1666 row = regionName[1];
1667 family = compaction.getFamilyName().toByteArray();
1668 isCompactionEntry = true;
1669 } catch (Exception ex) {
1670 LOG.warn("Unexpected exception received, ignoring " + ex);
1671 skippedCells.add(cell);
1672 continue;
1673 }
1674 } else {
1675 skippedCells.add(cell);
1676 continue;
1677 }
1678 }
1679
1680 try {
1681 loc =
1682 locateRegionAndRefreshLastFlushedSequenceId(hconn, table, row,
1683 encodeRegionNameStr);
1684
1685 if (isCompactionEntry && !encodeRegionNameStr.equalsIgnoreCase(
1686 loc.getRegionInfo().getEncodedName())) {
1687 LOG.info("Not replaying a compaction marker for an older region: "
1688 + encodeRegionNameStr);
1689 needSkip = true;
1690 }
1691 } catch (TableNotFoundException ex) {
1692
1693 LOG.info("Table " + table + " doesn't exist. Skip log replay for region "
1694 + encodeRegionNameStr);
1695 lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE);
1696 if (nonExistentTables == null) {
1697 nonExistentTables = new TreeSet<TableName>();
1698 }
1699 nonExistentTables.add(table);
1700 this.skippedEdits.incrementAndGet();
1701 needSkip = true;
1702 break;
1703 }
1704
1705 cachedLastFlushedSequenceId =
1706 lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
1707 if (cachedLastFlushedSequenceId != null
1708 && cachedLastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
1709
1710 this.skippedEdits.incrementAndGet();
1711 needSkip = true;
1712 break;
1713 } else {
1714 if (maxStoreSequenceIds == null) {
1715 maxStoreSequenceIds =
1716 regionMaxSeqIdInStores.get(loc.getRegionInfo().getEncodedName());
1717 }
1718 if (maxStoreSequenceIds != null) {
1719 Long maxStoreSeqId = maxStoreSequenceIds.get(family);
1720 if (maxStoreSeqId == null || maxStoreSeqId >= entry.getKey().getLogSeqNum()) {
1721
1722 skippedCells.add(cell);
1723 continue;
1724 }
1725 }
1726 }
1727 }
1728
1729
1730 if (loc == null || needSkip) continue;
1731
1732 if (!skippedCells.isEmpty()) {
1733 cells.removeAll(skippedCells);
1734 }
1735
1736 synchronized (serverToBufferQueueMap) {
1737 locKey = loc.getHostnamePort() + KEY_DELIMITER + table;
1738 List<Pair<HRegionLocation, Entry>> queue = serverToBufferQueueMap.get(locKey);
1739 if (queue == null) {
1740 queue =
1741 Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, Entry>>());
1742 serverToBufferQueueMap.put(locKey, queue);
1743 }
1744 queue.add(new Pair<HRegionLocation, Entry>(loc, entry));
1745 }
1746
1747 addToRecoveredRegions(loc.getRegionInfo().getEncodedName());
1748 }
1749 }
1750
1751
1752
1753
1754
1755
1756 private HRegionLocation locateRegionAndRefreshLastFlushedSequenceId(HConnection hconn,
1757 TableName table, byte[] row, String originalEncodedRegionName) throws IOException {
1758
1759 HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
1760 if(loc != null) return loc;
1761
1762 loc = hconn.getRegionLocation(table, row, true);
1763 if (loc == null) {
1764 throw new IOException("Can't locate location for row:" + Bytes.toString(row)
1765 + " of table:" + table);
1766 }
1767
1768 if (!originalEncodedRegionName.equalsIgnoreCase(loc.getRegionInfo().getEncodedName())) {
1769
1770 lastFlushedSequenceIds.put(originalEncodedRegionName, Long.MAX_VALUE);
1771 HRegionLocation tmpLoc = onlineRegions.get(loc.getRegionInfo().getEncodedName());
1772 if (tmpLoc != null) return tmpLoc;
1773 }
1774
1775 Long lastFlushedSequenceId = -1l;
1776 AtomicBoolean isRecovering = new AtomicBoolean(true);
1777 loc = waitUntilRegionOnline(loc, row, this.waitRegionOnlineTimeOut, isRecovering);
1778 if (!isRecovering.get()) {
1779
1780
1781 lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), Long.MAX_VALUE);
1782 LOG.info("logReplay skip region: " + loc.getRegionInfo().getEncodedName()
1783 + " because it's not in recovering.");
1784 } else {
1785 Long cachedLastFlushedSequenceId =
1786 lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
1787
1788
1789
1790 RegionStoreSequenceIds ids =
1791 csm.getSplitLogWorkerCoordination().getRegionFlushedSequenceId(failedServerName,
1792 loc.getRegionInfo().getEncodedName());
1793 if (ids != null) {
1794 lastFlushedSequenceId = ids.getLastFlushedSequenceId();
1795 Map<byte[], Long> storeIds = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1796 List<StoreSequenceId> maxSeqIdInStores = ids.getStoreSequenceIdList();
1797 for (StoreSequenceId id : maxSeqIdInStores) {
1798 storeIds.put(id.getFamilyName().toByteArray(), id.getSequenceId());
1799 }
1800 regionMaxSeqIdInStores.put(loc.getRegionInfo().getEncodedName(), storeIds);
1801 }
1802
1803 if (cachedLastFlushedSequenceId == null
1804 || lastFlushedSequenceId > cachedLastFlushedSequenceId) {
1805 lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), lastFlushedSequenceId);
1806 }
1807 }
1808
1809 onlineRegions.put(loc.getRegionInfo().getEncodedName(), loc);
1810 return loc;
1811 }
1812
1813 private void processWorkItems(String key, List<Pair<HRegionLocation, Entry>> actions)
1814 throws IOException {
1815 RegionServerWriter rsw = null;
1816
1817 long startTime = System.nanoTime();
1818 try {
1819 rsw = getRegionServerWriter(key);
1820 rsw.sink.replayEntries(actions);
1821
1822
1823 rsw.incrementEdits(actions.size());
1824 rsw.incrementNanoTime(System.nanoTime() - startTime);
1825 } catch (IOException e) {
1826 e = RemoteExceptionHandler.checkIOException(e);
1827 LOG.fatal(" Got while writing log entry to log", e);
1828 throw e;
1829 }
1830 }
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841 private HRegionLocation waitUntilRegionOnline(HRegionLocation loc, byte[] row,
1842 final long timeout, AtomicBoolean isRecovering)
1843 throws IOException {
1844 final long endTime = EnvironmentEdgeManager.currentTime() + timeout;
1845 final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
1846 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
1847 boolean reloadLocation = false;
1848 TableName tableName = loc.getRegionInfo().getTable();
1849 int tries = 0;
1850 Throwable cause = null;
1851 while (endTime > EnvironmentEdgeManager.currentTime()) {
1852 try {
1853
1854 HConnection hconn = getConnectionByTableName(tableName);
1855 if(reloadLocation) {
1856 loc = hconn.getRegionLocation(tableName, row, true);
1857 }
1858 BlockingInterface remoteSvr = hconn.getAdmin(loc.getServerName());
1859 HRegionInfo region = loc.getRegionInfo();
1860 try {
1861 GetRegionInfoRequest request =
1862 RequestConverter.buildGetRegionInfoRequest(region.getRegionName());
1863 GetRegionInfoResponse response = remoteSvr.getRegionInfo(null, request);
1864 if (HRegionInfo.convert(response.getRegionInfo()) != null) {
1865 isRecovering.set((response.hasIsRecovering()) ? response.getIsRecovering() : true);
1866 return loc;
1867 }
1868 } catch (ServiceException se) {
1869 throw ProtobufUtil.getRemoteException(se);
1870 }
1871 } catch (IOException e) {
1872 cause = e.getCause();
1873 if(!(cause instanceof RegionOpeningException)) {
1874 reloadLocation = true;
1875 }
1876 }
1877 long expectedSleep = ConnectionUtils.getPauseTime(pause, tries);
1878 try {
1879 Thread.sleep(expectedSleep);
1880 } catch (InterruptedException e) {
1881 throw new IOException("Interrupted when waiting region " +
1882 loc.getRegionInfo().getEncodedName() + " online.", e);
1883 }
1884 tries++;
1885 }
1886
1887 throw new IOException("Timeout when waiting region " + loc.getRegionInfo().getEncodedName() +
1888 " online for " + timeout + " milliseconds.", cause);
1889 }
1890
1891 @Override
1892 protected boolean flush() throws IOException {
1893 String curLoc = null;
1894 int curSize = 0;
1895 List<Pair<HRegionLocation, Entry>> curQueue = null;
1896 synchronized (this.serverToBufferQueueMap) {
1897 for (String locationKey : this.serverToBufferQueueMap.keySet()) {
1898 curQueue = this.serverToBufferQueueMap.get(locationKey);
1899 if (!curQueue.isEmpty()) {
1900 curSize = curQueue.size();
1901 curLoc = locationKey;
1902 break;
1903 }
1904 }
1905 if (curSize > 0) {
1906 this.serverToBufferQueueMap.remove(curLoc);
1907 }
1908 }
1909
1910 if (curSize > 0) {
1911 this.processWorkItems(curLoc, curQueue);
1912
1913 synchronized(dataAvailable) {
1914 dataAvailable.notifyAll();
1915 }
1916 return true;
1917 }
1918 return false;
1919 }
1920
1921 void addWriterError(Throwable t) {
1922 thrown.add(t);
1923 }
1924
1925 @Override
1926 List<Path> finishWritingAndClose() throws IOException {
1927 try {
1928 if (!finishWriting()) {
1929 return null;
1930 }
1931 if (hasEditsInDisablingOrDisabledTables) {
1932 splits = logRecoveredEditsOutputSink.finishWritingAndClose();
1933 } else {
1934 splits = new ArrayList<Path>();
1935 }
1936
1937 return splits;
1938 } finally {
1939 List<IOException> thrown = closeRegionServerWriters();
1940 if (thrown != null && !thrown.isEmpty()) {
1941 throw MultipleIOException.createIOException(thrown);
1942 }
1943 }
1944 }
1945
1946 @Override
1947 int getNumOpenWriters() {
1948 return this.writers.size() + this.logRecoveredEditsOutputSink.getNumOpenWriters();
1949 }
1950
1951 private List<IOException> closeRegionServerWriters() throws IOException {
1952 List<IOException> result = null;
1953 if (!writersClosed) {
1954 result = Lists.newArrayList();
1955 try {
1956 for (WriterThread t : writerThreads) {
1957 while (t.isAlive()) {
1958 t.shouldStop = true;
1959 t.interrupt();
1960 try {
1961 t.join(10);
1962 } catch (InterruptedException e) {
1963 IOException iie = new InterruptedIOException();
1964 iie.initCause(e);
1965 throw iie;
1966 }
1967 }
1968 }
1969 } finally {
1970 synchronized (writers) {
1971 for (String locationKey : writers.keySet()) {
1972 RegionServerWriter tmpW = writers.get(locationKey);
1973 try {
1974 tmpW.close();
1975 } catch (IOException ioe) {
1976 LOG.error("Couldn't close writer for region server:" + locationKey, ioe);
1977 result.add(ioe);
1978 }
1979 }
1980 }
1981
1982
1983 synchronized (this.tableNameToHConnectionMap) {
1984 for (TableName tableName : this.tableNameToHConnectionMap.keySet()) {
1985 HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
1986 try {
1987 hconn.clearRegionCache();
1988 hconn.close();
1989 } catch (IOException ioe) {
1990 result.add(ioe);
1991 }
1992 }
1993 }
1994 writersClosed = true;
1995 }
1996 }
1997 return result;
1998 }
1999
2000 @Override
2001 Map<byte[], Long> getOutputCounts() {
2002 TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2003 synchronized (writers) {
2004 for (Map.Entry<String, RegionServerWriter> entry : writers.entrySet()) {
2005 ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
2006 }
2007 }
2008 return ret;
2009 }
2010
2011 @Override
2012 int getNumberOfRecoveredRegions() {
2013 return this.recoveredRegions.size();
2014 }
2015
2016
2017
2018
2019
2020
2021 private RegionServerWriter getRegionServerWriter(String loc) throws IOException {
2022 RegionServerWriter ret = writers.get(loc);
2023 if (ret != null) {
2024 return ret;
2025 }
2026
2027 TableName tableName = getTableFromLocationStr(loc);
2028 if(tableName == null){
2029 throw new IOException("Invalid location string:" + loc + " found. Replay aborted.");
2030 }
2031
2032 HConnection hconn = getConnectionByTableName(tableName);
2033 synchronized (writers) {
2034 ret = writers.get(loc);
2035 if (ret == null) {
2036 ret = new RegionServerWriter(conf, tableName, hconn);
2037 writers.put(loc, ret);
2038 }
2039 }
2040 return ret;
2041 }
2042
2043 private HConnection getConnectionByTableName(final TableName tableName) throws IOException {
2044 HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
2045 if (hconn == null) {
2046 synchronized (this.tableNameToHConnectionMap) {
2047 hconn = this.tableNameToHConnectionMap.get(tableName);
2048 if (hconn == null) {
2049 hconn = HConnectionManager.getConnection(conf);
2050 this.tableNameToHConnectionMap.put(tableName, hconn);
2051 }
2052 }
2053 }
2054 return hconn;
2055 }
2056 private TableName getTableFromLocationStr(String loc) {
2057
2058
2059
2060 String[] splits = loc.split(KEY_DELIMITER);
2061 if (splits.length != 2) {
2062 return null;
2063 }
2064 return TableName.valueOf(splits[1]);
2065 }
2066 }
2067
2068
2069
2070
2071
2072 private final static class RegionServerWriter extends SinkWriter {
2073 final WALEditsReplaySink sink;
2074
2075 RegionServerWriter(final Configuration conf, final TableName tableName, final HConnection conn)
2076 throws IOException {
2077 this.sink = new WALEditsReplaySink(conf, tableName, conn);
2078 }
2079
2080 void close() throws IOException {
2081 }
2082 }
2083
2084 static class CorruptedLogFileException extends Exception {
2085 private static final long serialVersionUID = 1L;
2086
2087 CorruptedLogFileException(String s) {
2088 super(s);
2089 }
2090 }
2091
2092
2093 public static class MutationReplay {
2094 public MutationReplay(MutationType type, Mutation mutation, long nonceGroup, long nonce) {
2095 this.type = type;
2096 this.mutation = mutation;
2097 if(this.mutation.getDurability() != Durability.SKIP_WAL) {
2098
2099 this.mutation.setDurability(Durability.ASYNC_WAL);
2100 }
2101 this.nonceGroup = nonceGroup;
2102 this.nonce = nonce;
2103 }
2104
2105 public final MutationType type;
2106 public final Mutation mutation;
2107 public final long nonceGroup;
2108 public final long nonce;
2109 }
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121 public static List<MutationReplay> getMutationsFromWALEntry(WALEntry entry, CellScanner cells,
2122 Pair<WALKey, WALEdit> logEntry) throws IOException {
2123
2124 if (entry == null) {
2125
2126 return new ArrayList<MutationReplay>();
2127 }
2128
2129 long replaySeqId = (entry.getKey().hasOrigSequenceNumber()) ?
2130 entry.getKey().getOrigSequenceNumber() : entry.getKey().getLogSequenceNumber();
2131 int count = entry.getAssociatedCellCount();
2132 List<MutationReplay> mutations = new ArrayList<MutationReplay>();
2133 Cell previousCell = null;
2134 Mutation m = null;
2135 WALKey key = null;
2136 WALEdit val = null;
2137 if (logEntry != null) val = new WALEdit();
2138
2139 for (int i = 0; i < count; i++) {
2140
2141 if (!cells.advance()) {
2142 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
2143 }
2144 Cell cell = cells.current();
2145 if (val != null) val.add(cell);
2146
2147 boolean isNewRowOrType =
2148 previousCell == null || previousCell.getTypeByte() != cell.getTypeByte()
2149 || !CellUtil.matchingRow(previousCell, cell);
2150 if (isNewRowOrType) {
2151
2152 if (CellUtil.isDelete(cell)) {
2153 m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2154
2155 mutations.add(new MutationReplay(
2156 MutationType.DELETE, m, HConstants.NO_NONCE, HConstants.NO_NONCE));
2157 } else {
2158 m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
2159
2160 long nonceGroup = entry.getKey().hasNonceGroup()
2161 ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
2162 long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
2163 mutations.add(new MutationReplay(MutationType.PUT, m, nonceGroup, nonce));
2164 }
2165 }
2166 if (CellUtil.isDelete(cell)) {
2167 ((Delete) m).addDeleteMarker(cell);
2168 } else {
2169 ((Put) m).add(cell);
2170 }
2171 previousCell = cell;
2172 }
2173
2174
2175 if (logEntry != null) {
2176 org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey walKeyProto = entry.getKey();
2177 List<UUID> clusterIds = new ArrayList<UUID>(walKeyProto.getClusterIdsCount());
2178 for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
2179 clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
2180 }
2181
2182 key = new HLogKey(walKeyProto.getEncodedRegionName().toByteArray(), TableName.valueOf(
2183 walKeyProto.getTableName().toByteArray()), replaySeqId, walKeyProto.getWriteTime(),
2184 clusterIds, walKeyProto.getNonceGroup(), walKeyProto.getNonce());
2185 logEntry.setFirst(key);
2186 logEntry.setSecond(val);
2187 }
2188
2189 return mutations;
2190 }
2191 }