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 */ 018 019 package org.apache.hadoop.fs; 020 021 import java.io.*; 022 import java.util.Arrays; 023 import java.util.Iterator; 024 import java.util.zip.CRC32; 025 026 import org.apache.commons.logging.Log; 027 import org.apache.commons.logging.LogFactory; 028 import org.apache.hadoop.classification.InterfaceAudience; 029 import org.apache.hadoop.classification.InterfaceStability; 030 import org.apache.hadoop.conf.Configuration; 031 import org.apache.hadoop.fs.permission.FsPermission; 032 import org.apache.hadoop.util.Progressable; 033 import org.apache.hadoop.util.PureJavaCrc32; 034 import org.apache.hadoop.util.StringUtils; 035 036 /**************************************************************** 037 * Abstract Checksumed FileSystem. 038 * It provide a basice implementation of a Checksumed FileSystem, 039 * which creates a checksum file for each raw file. 040 * It generates & verifies checksums at the client side. 041 * 042 *****************************************************************/ 043 @InterfaceAudience.Public 044 @InterfaceStability.Stable 045 public abstract class ChecksumFileSystem extends FilterFileSystem { 046 private static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0}; 047 private int bytesPerChecksum = 512; 048 private boolean verifyChecksum = true; 049 050 public static double getApproxChkSumLength(long size) { 051 return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size; 052 } 053 054 public ChecksumFileSystem(FileSystem fs) { 055 super(fs); 056 } 057 058 public void setConf(Configuration conf) { 059 super.setConf(conf); 060 if (conf != null) { 061 bytesPerChecksum = conf.getInt(LocalFileSystemConfigKeys.LOCAL_FS_BYTES_PER_CHECKSUM_KEY, 062 LocalFileSystemConfigKeys.LOCAL_FS_BYTES_PER_CHECKSUM_DEFAULT); 063 } 064 } 065 066 /** 067 * Set whether to verify checksum. 068 */ 069 public void setVerifyChecksum(boolean verifyChecksum) { 070 this.verifyChecksum = verifyChecksum; 071 } 072 073 /** get the raw file system */ 074 public FileSystem getRawFileSystem() { 075 return fs; 076 } 077 078 /** Return the name of the checksum file associated with a file.*/ 079 public Path getChecksumFile(Path file) { 080 return new Path(file.getParent(), "." + file.getName() + ".crc"); 081 } 082 083 /** Return true iff file is a checksum file name.*/ 084 public static boolean isChecksumFile(Path file) { 085 String name = file.getName(); 086 return name.startsWith(".") && name.endsWith(".crc"); 087 } 088 089 /** Return the length of the checksum file given the size of the 090 * actual file. 091 **/ 092 public long getChecksumFileLength(Path file, long fileSize) { 093 return getChecksumLength(fileSize, getBytesPerSum()); 094 } 095 096 /** Return the bytes Per Checksum */ 097 public int getBytesPerSum() { 098 return bytesPerChecksum; 099 } 100 101 private int getSumBufferSize(int bytesPerSum, int bufferSize) { 102 int defaultBufferSize = getConf().getInt( 103 LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, 104 LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT); 105 int proportionalBufferSize = bufferSize / bytesPerSum; 106 return Math.max(bytesPerSum, 107 Math.max(proportionalBufferSize, defaultBufferSize)); 108 } 109 110 /******************************************************* 111 * For open()'s FSInputStream 112 * It verifies that data matches checksums. 113 *******************************************************/ 114 private static class ChecksumFSInputChecker extends FSInputChecker { 115 public static final Log LOG 116 = LogFactory.getLog(FSInputChecker.class); 117 118 private ChecksumFileSystem fs; 119 private FSDataInputStream datas; 120 private FSDataInputStream sums; 121 122 private static final int HEADER_LENGTH = 8; 123 124 private int bytesPerSum = 1; 125 private long fileLen = -1L; 126 127 public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file) 128 throws IOException { 129 this(fs, file, fs.getConf().getInt( 130 LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, 131 LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT)); 132 } 133 134 public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize) 135 throws IOException { 136 super( file, fs.getFileStatus(file).getReplication() ); 137 this.datas = fs.getRawFileSystem().open(file, bufferSize); 138 this.fs = fs; 139 Path sumFile = fs.getChecksumFile(file); 140 try { 141 int sumBufferSize = fs.getSumBufferSize(fs.getBytesPerSum(), bufferSize); 142 sums = fs.getRawFileSystem().open(sumFile, sumBufferSize); 143 144 byte[] version = new byte[CHECKSUM_VERSION.length]; 145 sums.readFully(version); 146 if (!Arrays.equals(version, CHECKSUM_VERSION)) 147 throw new IOException("Not a checksum file: "+sumFile); 148 this.bytesPerSum = sums.readInt(); 149 set(fs.verifyChecksum, new PureJavaCrc32(), bytesPerSum, 4); 150 } catch (FileNotFoundException e) { // quietly ignore 151 set(fs.verifyChecksum, null, 1, 0); 152 } catch (IOException e) { // loudly ignore 153 LOG.warn("Problem opening checksum file: "+ file + 154 ". Ignoring exception: " , e); 155 set(fs.verifyChecksum, null, 1, 0); 156 } 157 } 158 159 private long getChecksumFilePos( long dataPos ) { 160 return HEADER_LENGTH + 4*(dataPos/bytesPerSum); 161 } 162 163 protected long getChunkPosition( long dataPos ) { 164 return dataPos/bytesPerSum*bytesPerSum; 165 } 166 167 public int available() throws IOException { 168 return datas.available() + super.available(); 169 } 170 171 public int read(long position, byte[] b, int off, int len) 172 throws IOException { 173 // parameter check 174 if ((off | len | (off + len) | (b.length - (off + len))) < 0) { 175 throw new IndexOutOfBoundsException(); 176 } else if (len == 0) { 177 return 0; 178 } 179 if( position<0 ) { 180 throw new IllegalArgumentException( 181 "Parameter position can not to be negative"); 182 } 183 184 ChecksumFSInputChecker checker = new ChecksumFSInputChecker(fs, file); 185 checker.seek(position); 186 int nread = checker.read(b, off, len); 187 checker.close(); 188 return nread; 189 } 190 191 public void close() throws IOException { 192 datas.close(); 193 if( sums != null ) { 194 sums.close(); 195 } 196 set(fs.verifyChecksum, null, 1, 0); 197 } 198 199 200 @Override 201 public boolean seekToNewSource(long targetPos) throws IOException { 202 long sumsPos = getChecksumFilePos(targetPos); 203 fs.reportChecksumFailure(file, datas, targetPos, sums, sumsPos); 204 boolean newDataSource = datas.seekToNewSource(targetPos); 205 return sums.seekToNewSource(sumsPos) || newDataSource; 206 } 207 208 @Override 209 protected int readChunk(long pos, byte[] buf, int offset, int len, 210 byte[] checksum) throws IOException { 211 212 boolean eof = false; 213 if (needChecksum()) { 214 assert checksum != null; // we have a checksum buffer 215 assert checksum.length % CHECKSUM_SIZE == 0; // it is sane length 216 assert len >= bytesPerSum; // we must read at least one chunk 217 218 final int checksumsToRead = Math.min( 219 len/bytesPerSum, // number of checksums based on len to read 220 checksum.length / CHECKSUM_SIZE); // size of checksum buffer 221 long checksumPos = getChecksumFilePos(pos); 222 if(checksumPos != sums.getPos()) { 223 sums.seek(checksumPos); 224 } 225 226 int sumLenRead = sums.read(checksum, 0, CHECKSUM_SIZE * checksumsToRead); 227 if (sumLenRead >= 0 && sumLenRead % CHECKSUM_SIZE != 0) { 228 throw new ChecksumException( 229 "Checksum file not a length multiple of checksum size " + 230 "in " + file + " at " + pos + " checksumpos: " + checksumPos + 231 " sumLenread: " + sumLenRead, 232 pos); 233 } 234 if (sumLenRead <= 0) { // we're at the end of the file 235 eof = true; 236 } else { 237 // Adjust amount of data to read based on how many checksum chunks we read 238 len = Math.min(len, bytesPerSum * (sumLenRead / CHECKSUM_SIZE)); 239 } 240 } 241 if(pos != datas.getPos()) { 242 datas.seek(pos); 243 } 244 int nread = readFully(datas, buf, offset, len); 245 if (eof && nread > 0) { 246 throw new ChecksumException("Checksum error: "+file+" at "+pos, pos); 247 } 248 return nread; 249 } 250 251 /* Return the file length */ 252 private long getFileLength() throws IOException { 253 if( fileLen==-1L ) { 254 fileLen = fs.getContentSummary(file).getLength(); 255 } 256 return fileLen; 257 } 258 259 /** 260 * Skips over and discards <code>n</code> bytes of data from the 261 * input stream. 262 * 263 *The <code>skip</code> method skips over some smaller number of bytes 264 * when reaching end of file before <code>n</code> bytes have been skipped. 265 * The actual number of bytes skipped is returned. If <code>n</code> is 266 * negative, no bytes are skipped. 267 * 268 * @param n the number of bytes to be skipped. 269 * @return the actual number of bytes skipped. 270 * @exception IOException if an I/O error occurs. 271 * ChecksumException if the chunk to skip to is corrupted 272 */ 273 public synchronized long skip(long n) throws IOException { 274 long curPos = getPos(); 275 long fileLength = getFileLength(); 276 if( n+curPos > fileLength ) { 277 n = fileLength - curPos; 278 } 279 return super.skip(n); 280 } 281 282 /** 283 * Seek to the given position in the stream. 284 * The next read() will be from that position. 285 * 286 * <p>This method does not allow seek past the end of the file. 287 * This produces IOException. 288 * 289 * @param pos the postion to seek to. 290 * @exception IOException if an I/O error occurs or seeks after EOF 291 * ChecksumException if the chunk to seek to is corrupted 292 */ 293 294 public synchronized void seek(long pos) throws IOException { 295 if(pos>getFileLength()) { 296 throw new IOException("Cannot seek after EOF"); 297 } 298 super.seek(pos); 299 } 300 301 } 302 303 /** 304 * Opens an FSDataInputStream at the indicated Path. 305 * @param f the file name to open 306 * @param bufferSize the size of the buffer to be used. 307 */ 308 @Override 309 public FSDataInputStream open(Path f, int bufferSize) throws IOException { 310 return new FSDataInputStream( 311 new ChecksumFSInputChecker(this, f, bufferSize)); 312 } 313 314 /** {@inheritDoc} */ 315 public FSDataOutputStream append(Path f, int bufferSize, 316 Progressable progress) throws IOException { 317 throw new IOException("Not supported"); 318 } 319 320 /** 321 * Calculated the length of the checksum file in bytes. 322 * @param size the length of the data file in bytes 323 * @param bytesPerSum the number of bytes in a checksum block 324 * @return the number of bytes in the checksum file 325 */ 326 public static long getChecksumLength(long size, int bytesPerSum) { 327 //the checksum length is equal to size passed divided by bytesPerSum + 328 //bytes written in the beginning of the checksum file. 329 return ((size + bytesPerSum - 1) / bytesPerSum) * 4 + 330 CHECKSUM_VERSION.length + 4; 331 } 332 333 /** This class provides an output stream for a checksummed file. 334 * It generates checksums for data. */ 335 private static class ChecksumFSOutputSummer extends FSOutputSummer { 336 private FSDataOutputStream datas; 337 private FSDataOutputStream sums; 338 private static final float CHKSUM_AS_FRACTION = 0.01f; 339 340 public ChecksumFSOutputSummer(ChecksumFileSystem fs, 341 Path file, 342 boolean overwrite, 343 short replication, 344 long blockSize, 345 Configuration conf) 346 throws IOException { 347 this(fs, file, overwrite, 348 conf.getInt(LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, 349 LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT), 350 replication, blockSize, null); 351 } 352 353 public ChecksumFSOutputSummer(ChecksumFileSystem fs, 354 Path file, 355 boolean overwrite, 356 int bufferSize, 357 short replication, 358 long blockSize, 359 Progressable progress) 360 throws IOException { 361 super(new PureJavaCrc32(), fs.getBytesPerSum(), 4); 362 int bytesPerSum = fs.getBytesPerSum(); 363 this.datas = fs.getRawFileSystem().create(file, overwrite, bufferSize, 364 replication, blockSize, progress); 365 int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize); 366 this.sums = fs.getRawFileSystem().create(fs.getChecksumFile(file), true, 367 sumBufferSize, replication, 368 blockSize); 369 sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length); 370 sums.writeInt(bytesPerSum); 371 } 372 373 public void close() throws IOException { 374 flushBuffer(); 375 sums.close(); 376 datas.close(); 377 } 378 379 @Override 380 protected void writeChunk(byte[] b, int offset, int len, byte[] checksum) 381 throws IOException { 382 datas.write(b, offset, len); 383 sums.write(checksum); 384 } 385 } 386 387 /** {@inheritDoc} */ 388 @Override 389 public FSDataOutputStream create(Path f, FsPermission permission, 390 boolean overwrite, int bufferSize, short replication, long blockSize, 391 Progressable progress) throws IOException { 392 Path parent = f.getParent(); 393 if (parent != null && !mkdirs(parent)) { 394 throw new IOException("Mkdirs failed to create " + parent); 395 } 396 final FSDataOutputStream out = new FSDataOutputStream( 397 new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication, 398 blockSize, progress), null); 399 if (permission != null) { 400 setPermission(f, permission); 401 } 402 return out; 403 } 404 405 /** 406 * Set replication for an existing file. 407 * Implement the abstract <tt>setReplication</tt> of <tt>FileSystem</tt> 408 * @param src file name 409 * @param replication new replication 410 * @throws IOException 411 * @return true if successful; 412 * false if file does not exist or is a directory 413 */ 414 public boolean setReplication(Path src, short replication) throws IOException { 415 boolean value = fs.setReplication(src, replication); 416 if (!value) 417 return false; 418 419 Path checkFile = getChecksumFile(src); 420 if (exists(checkFile)) 421 fs.setReplication(checkFile, replication); 422 423 return true; 424 } 425 426 /** 427 * Rename files/dirs 428 */ 429 public boolean rename(Path src, Path dst) throws IOException { 430 if (fs.isDirectory(src)) { 431 return fs.rename(src, dst); 432 } else { 433 434 boolean value = fs.rename(src, dst); 435 if (!value) 436 return false; 437 438 Path checkFile = getChecksumFile(src); 439 if (fs.exists(checkFile)) { //try to rename checksum 440 if (fs.isDirectory(dst)) { 441 value = fs.rename(checkFile, dst); 442 } else { 443 value = fs.rename(checkFile, getChecksumFile(dst)); 444 } 445 } 446 447 return value; 448 } 449 } 450 451 /** 452 * Implement the delete(Path, boolean) in checksum 453 * file system. 454 */ 455 public boolean delete(Path f, boolean recursive) throws IOException{ 456 FileStatus fstatus = null; 457 try { 458 fstatus = fs.getFileStatus(f); 459 } catch(FileNotFoundException e) { 460 return false; 461 } 462 if (fstatus.isDirectory()) { 463 //this works since the crcs are in the same 464 //directories and the files. so we just delete 465 //everything in the underlying filesystem 466 return fs.delete(f, recursive); 467 } else { 468 Path checkFile = getChecksumFile(f); 469 if (fs.exists(checkFile)) { 470 fs.delete(checkFile, true); 471 } 472 return fs.delete(f, true); 473 } 474 } 475 476 final private static PathFilter DEFAULT_FILTER = new PathFilter() { 477 public boolean accept(Path file) { 478 return !isChecksumFile(file); 479 } 480 }; 481 482 /** 483 * List the statuses of the files/directories in the given path if the path is 484 * a directory. 485 * 486 * @param f 487 * given path 488 * @return the statuses of the files/directories in the given patch 489 * @throws IOException 490 */ 491 @Override 492 public FileStatus[] listStatus(Path f) throws IOException { 493 return fs.listStatus(f, DEFAULT_FILTER); 494 } 495 496 /** 497 * List the statuses of the files/directories in the given path if the path is 498 * a directory. 499 * 500 * @param f 501 * given path 502 * @return the statuses of the files/directories in the given patch 503 * @throws IOException 504 */ 505 @Override 506 public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) 507 throws IOException { 508 return fs.listLocatedStatus(f, DEFAULT_FILTER); 509 } 510 511 @Override 512 public boolean mkdirs(Path f) throws IOException { 513 return fs.mkdirs(f); 514 } 515 516 @Override 517 public void copyFromLocalFile(boolean delSrc, Path src, Path dst) 518 throws IOException { 519 Configuration conf = getConf(); 520 FileUtil.copy(getLocal(conf), src, this, dst, delSrc, conf); 521 } 522 523 /** 524 * The src file is under FS, and the dst is on the local disk. 525 * Copy it from FS control to the local dst name. 526 */ 527 @Override 528 public void copyToLocalFile(boolean delSrc, Path src, Path dst) 529 throws IOException { 530 Configuration conf = getConf(); 531 FileUtil.copy(this, src, getLocal(conf), dst, delSrc, conf); 532 } 533 534 /** 535 * The src file is under FS, and the dst is on the local disk. 536 * Copy it from FS control to the local dst name. 537 * If src and dst are directories, the copyCrc parameter 538 * determines whether to copy CRC files. 539 */ 540 public void copyToLocalFile(Path src, Path dst, boolean copyCrc) 541 throws IOException { 542 if (!fs.isDirectory(src)) { // source is a file 543 fs.copyToLocalFile(src, dst); 544 FileSystem localFs = getLocal(getConf()).getRawFileSystem(); 545 if (localFs.isDirectory(dst)) { 546 dst = new Path(dst, src.getName()); 547 } 548 dst = getChecksumFile(dst); 549 if (localFs.exists(dst)) { //remove old local checksum file 550 localFs.delete(dst, true); 551 } 552 Path checksumFile = getChecksumFile(src); 553 if (copyCrc && fs.exists(checksumFile)) { //copy checksum file 554 fs.copyToLocalFile(checksumFile, dst); 555 } 556 } else { 557 FileStatus[] srcs = listStatus(src); 558 for (FileStatus srcFile : srcs) { 559 copyToLocalFile(srcFile.getPath(), 560 new Path(dst, srcFile.getPath().getName()), copyCrc); 561 } 562 } 563 } 564 565 @Override 566 public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) 567 throws IOException { 568 return tmpLocalFile; 569 } 570 571 @Override 572 public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) 573 throws IOException { 574 moveFromLocalFile(tmpLocalFile, fsOutputFile); 575 } 576 577 /** 578 * Report a checksum error to the file system. 579 * @param f the file name containing the error 580 * @param in the stream open on the file 581 * @param inPos the position of the beginning of the bad data in the file 582 * @param sums the stream open on the checksum file 583 * @param sumsPos the position of the beginning of the bad data in the checksum file 584 * @return if retry is neccessary 585 */ 586 public boolean reportChecksumFailure(Path f, FSDataInputStream in, 587 long inPos, FSDataInputStream sums, long sumsPos) { 588 return false; 589 } 590 }