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.mapreduce; 019 020import java.io.DataInput; 021import java.io.DataOutput; 022import java.io.EOFException; 023import java.io.FileNotFoundException; 024import java.io.IOException; 025import java.time.Instant; 026import java.util.ArrayList; 027import java.util.Collections; 028import java.util.List; 029import org.apache.hadoop.conf.Configuration; 030import org.apache.hadoop.fs.FileStatus; 031import org.apache.hadoop.fs.FileSystem; 032import org.apache.hadoop.fs.LocatedFileStatus; 033import org.apache.hadoop.fs.Path; 034import org.apache.hadoop.fs.RemoteIterator; 035import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; 036import org.apache.hadoop.hbase.wal.WAL; 037import org.apache.hadoop.hbase.wal.WAL.Entry; 038import org.apache.hadoop.hbase.wal.WAL.Reader; 039import org.apache.hadoop.hbase.wal.WALEdit; 040import org.apache.hadoop.hbase.wal.WALKey; 041import org.apache.hadoop.io.Writable; 042import org.apache.hadoop.mapreduce.InputFormat; 043import org.apache.hadoop.mapreduce.InputSplit; 044import org.apache.hadoop.mapreduce.JobContext; 045import org.apache.hadoop.mapreduce.RecordReader; 046import org.apache.hadoop.mapreduce.TaskAttemptContext; 047import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; 048import org.apache.hadoop.mapreduce.security.TokenCache; 049import org.apache.hadoop.util.StringUtils; 050import org.apache.yetus.audience.InterfaceAudience; 051import org.slf4j.Logger; 052import org.slf4j.LoggerFactory; 053 054/** 055 * Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} files. 056 */ 057@InterfaceAudience.Public 058public class WALInputFormat extends InputFormat<WALKey, WALEdit> { 059 private static final Logger LOG = LoggerFactory.getLogger(WALInputFormat.class); 060 061 public static final String START_TIME_KEY = "wal.start.time"; 062 public static final String END_TIME_KEY = "wal.end.time"; 063 064 /** 065 * {@link InputSplit} for {@link WAL} files. Each split represent 066 * exactly one log file. 067 */ 068 static class WALSplit extends InputSplit implements Writable { 069 private String logFileName; 070 private long fileSize; 071 private long startTime; 072 private long endTime; 073 074 /** for serialization */ 075 public WALSplit() {} 076 077 /** 078 * Represent an WALSplit, i.e. a single WAL file. 079 * Start- and EndTime are managed by the split, so that WAL files can be 080 * filtered before WALEdits are passed to the mapper(s). 081 */ 082 public WALSplit(String logFileName, long fileSize, long startTime, long endTime) { 083 this.logFileName = logFileName; 084 this.fileSize = fileSize; 085 this.startTime = startTime; 086 this.endTime = endTime; 087 } 088 089 @Override 090 public long getLength() throws IOException, InterruptedException { 091 return fileSize; 092 } 093 094 @Override 095 public String[] getLocations() throws IOException, InterruptedException { 096 // TODO: Find the data node with the most blocks for this WAL? 097 return new String[] {}; 098 } 099 100 public String getLogFileName() { 101 return logFileName; 102 } 103 104 public long getStartTime() { 105 return startTime; 106 } 107 108 public long getEndTime() { 109 return endTime; 110 } 111 112 @Override 113 public void readFields(DataInput in) throws IOException { 114 logFileName = in.readUTF(); 115 fileSize = in.readLong(); 116 startTime = in.readLong(); 117 endTime = in.readLong(); 118 } 119 120 @Override 121 public void write(DataOutput out) throws IOException { 122 out.writeUTF(logFileName); 123 out.writeLong(fileSize); 124 out.writeLong(startTime); 125 out.writeLong(endTime); 126 } 127 128 @Override 129 public String toString() { 130 return logFileName + " (" + startTime + ":" + endTime + ") length:" + fileSize; 131 } 132 } 133 134 /** 135 * {@link RecordReader} for an {@link WAL} file. 136 * Implementation shared with deprecated HLogInputFormat. 137 */ 138 static abstract class WALRecordReader<K extends WALKey> extends RecordReader<K, WALEdit> { 139 private Reader reader = null; 140 // visible until we can remove the deprecated HLogInputFormat 141 Entry currentEntry = new Entry(); 142 private long startTime; 143 private long endTime; 144 private Configuration conf; 145 private Path logFile; 146 private long currentPos; 147 148 @Override 149 public void initialize(InputSplit split, TaskAttemptContext context) 150 throws IOException, InterruptedException { 151 WALSplit hsplit = (WALSplit)split; 152 logFile = new Path(hsplit.getLogFileName()); 153 conf = context.getConfiguration(); 154 LOG.info("Opening {} for {}", logFile, split); 155 openReader(logFile); 156 this.startTime = hsplit.getStartTime(); 157 this.endTime = hsplit.getEndTime(); 158 } 159 160 private void openReader(Path path) throws IOException { 161 closeReader(); 162 reader = AbstractFSWALProvider.openReader(path, conf); 163 seek(); 164 setCurrentPath(path); 165 } 166 167 private void setCurrentPath(Path path) { 168 this.logFile = path; 169 } 170 171 private void closeReader() throws IOException { 172 if (reader != null) { 173 reader.close(); 174 reader = null; 175 } 176 } 177 178 private void seek() throws IOException { 179 if (currentPos != 0) { 180 reader.seek(currentPos); 181 } 182 } 183 184 @Override 185 public boolean nextKeyValue() throws IOException, InterruptedException { 186 if (reader == null) { 187 return false; 188 } 189 this.currentPos = reader.getPosition(); 190 Entry temp; 191 long i = -1; 192 try { 193 do { 194 // skip older entries 195 try { 196 temp = reader.next(currentEntry); 197 i++; 198 } catch (EOFException x) { 199 LOG.warn("Corrupted entry detected. Ignoring the rest of the file." 200 + " (This is normal when a RegionServer crashed.)"); 201 return false; 202 } 203 } while (temp != null && temp.getKey().getWriteTime() < startTime); 204 205 if (temp == null) { 206 if (i > 0) { 207 LOG.info("Skipped " + i + " entries."); 208 } 209 LOG.info("Reached end of file."); 210 return false; 211 } else if (i > 0) { 212 LOG.info("Skipped " + i + " entries, until ts: " + temp.getKey().getWriteTime() + "."); 213 } 214 boolean res = temp.getKey().getWriteTime() <= endTime; 215 if (!res) { 216 LOG.info("Reached ts: " + temp.getKey().getWriteTime() 217 + " ignoring the rest of the file."); 218 } 219 return res; 220 } catch (IOException e) { 221 Path archivedLog = AbstractFSWALProvider.findArchivedLog(logFile, conf); 222 // archivedLog can be null if unable to locate in archiveDir. 223 if (archivedLog != null) { 224 openReader(archivedLog); 225 // Try call again in recursion 226 return nextKeyValue(); 227 } else { 228 throw e; 229 } 230 } 231 } 232 233 @Override 234 public WALEdit getCurrentValue() throws IOException, InterruptedException { 235 return currentEntry.getEdit(); 236 } 237 238 @Override 239 public float getProgress() throws IOException, InterruptedException { 240 // N/A depends on total number of entries, which is unknown 241 return 0; 242 } 243 244 @Override 245 public void close() throws IOException { 246 LOG.info("Closing reader"); 247 if (reader != null) { 248 this.reader.close(); 249 } 250 } 251 } 252 253 /** 254 * handler for non-deprecated WALKey version. fold into WALRecordReader once we no longer 255 * need to support HLogInputFormat. 256 */ 257 static class WALKeyRecordReader extends WALRecordReader<WALKey> { 258 @Override 259 public WALKey getCurrentKey() throws IOException, InterruptedException { 260 return currentEntry.getKey(); 261 } 262 } 263 264 @Override 265 public List<InputSplit> getSplits(JobContext context) throws IOException, 266 InterruptedException { 267 return getSplits(context, START_TIME_KEY, END_TIME_KEY); 268 } 269 270 /** 271 * implementation shared with deprecated HLogInputFormat 272 */ 273 List<InputSplit> getSplits(final JobContext context, final String startKey, final String endKey) 274 throws IOException, InterruptedException { 275 Configuration conf = context.getConfiguration(); 276 boolean ignoreMissing = conf.getBoolean(WALPlayer.IGNORE_MISSING_FILES, false); 277 Path[] inputPaths = getInputPaths(conf); 278 // get delegation token for the filesystem 279 TokenCache.obtainTokensForNamenodes(context.getCredentials(), inputPaths, conf); 280 long startTime = conf.getLong(startKey, Long.MIN_VALUE); 281 long endTime = conf.getLong(endKey, Long.MAX_VALUE); 282 283 List<FileStatus> allFiles = new ArrayList<FileStatus>(); 284 for(Path inputPath: inputPaths){ 285 FileSystem fs = inputPath.getFileSystem(conf); 286 try { 287 List<FileStatus> files = getFiles(fs, inputPath, startTime, endTime); 288 allFiles.addAll(files); 289 } catch (FileNotFoundException e) { 290 if (ignoreMissing) { 291 LOG.warn("File "+ inputPath +" is missing. Skipping it."); 292 continue; 293 } 294 throw e; 295 } 296 } 297 List<InputSplit> splits = new ArrayList<InputSplit>(allFiles.size()); 298 for (FileStatus file : allFiles) { 299 splits.add(new WALSplit(file.getPath().toString(), file.getLen(), startTime, endTime)); 300 } 301 return splits; 302 } 303 304 private Path[] getInputPaths(Configuration conf) { 305 String inpDirs = conf.get(FileInputFormat.INPUT_DIR); 306 return StringUtils.stringToPath( 307 inpDirs.split(conf.get(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ","))); 308 } 309 310 /** 311 * @param startTime If file looks like it has a timestamp in its name, we'll check if newer 312 * or equal to this value else we will filter out the file. If name does not 313 * seem to have a timestamp, we will just return it w/o filtering. 314 * @param endTime If file looks like it has a timestamp in its name, we'll check if older or equal 315 * to this value else we will filter out the file. If name does not seem to 316 * have a timestamp, we will just return it w/o filtering. 317 */ 318 private List<FileStatus> getFiles(FileSystem fs, Path dir, long startTime, long endTime) 319 throws IOException { 320 List<FileStatus> result = new ArrayList<>(); 321 LOG.debug("Scanning " + dir.toString() + " for WAL files"); 322 RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(dir); 323 if (!iter.hasNext()) { 324 return Collections.emptyList(); 325 } 326 while (iter.hasNext()) { 327 LocatedFileStatus file = iter.next(); 328 if (file.isDirectory()) { 329 // Recurse into sub directories 330 result.addAll(getFiles(fs, file.getPath(), startTime, endTime)); 331 } else { 332 addFile(result, file, startTime, endTime); 333 } 334 } 335 // TODO: These results should be sorted? Results could be content of recovered.edits directory 336 // -- null padded increasing numeric -- or a WAL file w/ timestamp suffix or timestamp and 337 // then meta suffix. See AbstractFSWALProvider#WALStartTimeComparator 338 return result; 339 } 340 341 static void addFile(List<FileStatus> result, LocatedFileStatus lfs, long startTime, 342 long endTime) { 343 long timestamp = WAL.getTimestamp(lfs.getPath().getName()); 344 if (timestamp > 0) { 345 // Looks like a valid timestamp. 346 if (timestamp <= endTime && timestamp >= startTime) { 347 LOG.info("Found {}", lfs.getPath()); 348 result.add(lfs); 349 } else { 350 LOG.info("Skipped {}, outside range [{}/{} - {}/{}]", lfs.getPath(), 351 startTime, Instant.ofEpochMilli(startTime), endTime, Instant.ofEpochMilli(endTime)); 352 } 353 } else { 354 // If no timestamp, add it regardless. 355 LOG.info("Found (no-timestamp!) {}", lfs); 356 result.add(lfs); 357 } 358 } 359 360 @Override 361 public RecordReader<WALKey, WALEdit> createRecordReader(InputSplit split, 362 TaskAttemptContext context) throws IOException, InterruptedException { 363 return new WALKeyRecordReader(); 364 } 365}