@@ -1084,27 +1084,44 @@ public HStoreFile tryCommitRecoveredHFile(Path path) throws IOException {
10841084 }
10851085
10861086 /**
1087- * @param path The pathname of the tmp file into which the store was flushed
1088- * @return store file created.
1087+ * Commit the given {@code files}.
1088+ * <p/>
1089+ * We will move the file into data directory, and open it.
1090+ * @param files the files want to commit
1091+ * @param validate whether to validate the store files
1092+ * @return the committed store files
10891093 */
1090- private HStoreFile commitFile (Path path , long logCacheFlushId , MonitoredTask status )
1091- throws IOException {
1092- // Write-out finished successfully, move into the right spot
1093- Path dstPath = getRegionFileSystem ().commitStoreFile (getColumnFamilyName (), path );
1094-
1095- status .setStatus ("Flushing " + this + ": reopening flushed file" );
1096- HStoreFile sf = createStoreFileAndReader (dstPath );
1097-
1098- StoreFileReader r = sf .getReader ();
1099- this .storeSize .addAndGet (r .length ());
1100- this .totalUncompressedBytes .addAndGet (r .getTotalUncompressedBytes ());
1101-
1102- if (LOG .isInfoEnabled ()) {
1103- LOG .info ("Added " + sf + ", entries=" + r .getEntries () +
1104- ", sequenceid=" + logCacheFlushId +
1105- ", filesize=" + TraditionalBinaryPrefix .long2String (r .length (), "" , 1 ));
1094+ private List <HStoreFile > commitStoreFiles (List <Path > files , boolean validate ) throws IOException {
1095+ List <HStoreFile > committedFiles = new ArrayList <>(files .size ());
1096+ HRegionFileSystem hfs = getRegionFileSystem ();
1097+ String familyName = getColumnFamilyName ();
1098+ for (Path file : files ) {
1099+ try {
1100+ if (validate ) {
1101+ validateStoreFile (file );
1102+ }
1103+ Path committedPath = hfs .commitStoreFile (familyName , file );
1104+ HStoreFile sf = createStoreFileAndReader (committedPath );
1105+ committedFiles .add (sf );
1106+ } catch (IOException e ) {
1107+ LOG .error ("Failed to commit store file {}" , file , e );
1108+ // Try to delete the files we have committed before.
1109+ // It is OK to fail when deleting as leaving the file there does not cause any data
1110+ // corruption problem. It just introduces some duplicated data which may impact read
1111+ // performance a little when reading before compaction.
1112+ for (HStoreFile sf : committedFiles ) {
1113+ Path pathToDelete = sf .getPath ();
1114+ try {
1115+ sf .deleteStoreFile ();
1116+ } catch (IOException deleteEx ) {
1117+ LOG .warn (HBaseMarkers .FATAL , "Failed to delete committed store file {}" , pathToDelete ,
1118+ deleteEx );
1119+ }
1120+ }
1121+ throw new IOException ("Failed to commit the flush" , e );
1122+ }
11061123 }
1107- return sf ;
1124+ return committedFiles ;
11081125 }
11091126
11101127 public StoreFileWriter createWriterInTmp (long maxKeyCount , Compression .Algorithm compression ,
@@ -1501,7 +1518,12 @@ protected List<HStoreFile> doCompaction(CompactionRequestImpl cr,
15011518 List <Path > newFiles ) throws IOException {
15021519 // Do the steps necessary to complete the compaction.
15031520 setStoragePolicyFromFileName (newFiles );
1504- List <HStoreFile > sfs = moveCompactedFilesIntoPlace (cr , newFiles , user );
1521+ List <HStoreFile > sfs = commitStoreFiles (newFiles , true );
1522+ if (this .getCoprocessorHost () != null ) {
1523+ for (HStoreFile sf : sfs ) {
1524+ getCoprocessorHost ().postCompact (this , sf , cr .getTracker (), cr , user );
1525+ }
1526+ }
15051527 writeCompactionWalRecord (filesToCompact , sfs );
15061528 replaceStoreFiles (filesToCompact , sfs );
15071529 if (cr .isMajor ()) {
@@ -1542,29 +1564,6 @@ private void setStoragePolicyFromFileName(List<Path> newFiles) throws IOExceptio
15421564 }
15431565 }
15441566
1545- private List <HStoreFile > moveCompactedFilesIntoPlace (CompactionRequestImpl cr ,
1546- List <Path > newFiles , User user ) throws IOException {
1547- List <HStoreFile > sfs = new ArrayList <>(newFiles .size ());
1548- for (Path newFile : newFiles ) {
1549- assert newFile != null ;
1550- HStoreFile sf = moveFileIntoPlace (newFile );
1551- if (this .getCoprocessorHost () != null ) {
1552- getCoprocessorHost ().postCompact (this , sf , cr .getTracker (), cr , user );
1553- }
1554- assert sf != null ;
1555- sfs .add (sf );
1556- }
1557- return sfs ;
1558- }
1559-
1560- // Package-visible for tests
1561- HStoreFile moveFileIntoPlace (Path newFile ) throws IOException {
1562- validateStoreFile (newFile );
1563- // Move the file into the right spot
1564- Path destPath = getRegionFileSystem ().commitStoreFile (getColumnFamilyName (), newFile );
1565- return createStoreFileAndReader (destPath );
1566- }
1567-
15681567 /**
15691568 * Writes the compaction WAL record.
15701569 * @param filesCompacted Files compacted (input).
@@ -2346,42 +2345,31 @@ public boolean commit(MonitoredTask status) throws IOException {
23462345 if (CollectionUtils .isEmpty (this .tempFiles )) {
23472346 return false ;
23482347 }
2349- List <HStoreFile > storeFiles = new ArrayList <>(this .tempFiles .size ());
2350- for (Path storeFilePath : tempFiles ) {
2351- try {
2352- HStoreFile sf = HStore .this .commitFile (storeFilePath , cacheFlushSeqNum , status );
2353- outputFileSize += sf .getReader ().length ();
2354- storeFiles .add (sf );
2355- } catch (IOException ex ) {
2356- LOG .error ("Failed to commit store file {}" , storeFilePath , ex );
2357- // Try to delete the files we have committed before.
2358- for (HStoreFile sf : storeFiles ) {
2359- Path pathToDelete = sf .getPath ();
2360- try {
2361- sf .deleteStoreFile ();
2362- } catch (IOException deleteEx ) {
2363- LOG .error (HBaseMarkers .FATAL , "Failed to delete store file we committed, "
2364- + "halting {}" , pathToDelete , ex );
2365- Runtime .getRuntime ().halt (1 );
2366- }
2367- }
2368- throw new IOException ("Failed to commit the flush" , ex );
2348+ status .setStatus ("Flushing " + this + ": reopening flushed file" );
2349+ List <HStoreFile > storeFiles = commitStoreFiles (tempFiles , false );
2350+ for (HStoreFile sf : storeFiles ) {
2351+ StoreFileReader r = sf .getReader ();
2352+ if (LOG .isInfoEnabled ()) {
2353+ LOG .info ("Added {}, entries={}, sequenceid={}, filesize={}" , sf , r .getEntries (),
2354+ cacheFlushSeqNum , TraditionalBinaryPrefix .long2String (r .length (), "" , 1 ));
23692355 }
2356+ outputFileSize += r .length ();
2357+ storeSize .addAndGet (r .length ());
2358+ totalUncompressedBytes .addAndGet (r .getTotalUncompressedBytes ());
2359+ committedFiles .add (sf .getPath ());
23702360 }
23712361
2362+ flushedCellsCount .addAndGet (cacheFlushCount );
2363+ flushedCellsSize .addAndGet (cacheFlushSize );
2364+ flushedOutputFileSize .addAndGet (outputFileSize );
2365+ // call coprocessor after we have done all the accounting above
23722366 for (HStoreFile sf : storeFiles ) {
2373- if (HStore . this . getCoprocessorHost () != null ) {
2374- HStore . this . getCoprocessorHost ().postFlush (HStore .this , sf , tracker );
2367+ if (getCoprocessorHost () != null ) {
2368+ getCoprocessorHost ().postFlush (HStore .this , sf , tracker );
23752369 }
2376- committedFiles .add (sf .getPath ());
23772370 }
2378-
2379- HStore .this .flushedCellsCount .addAndGet (cacheFlushCount );
2380- HStore .this .flushedCellsSize .addAndGet (cacheFlushSize );
2381- HStore .this .flushedOutputFileSize .addAndGet (outputFileSize );
2382-
2383- // Add new file to store files. Clear snapshot too while we have the Store write lock.
2384- return HStore .this .updateStorefiles (storeFiles , snapshot .getId ());
2371+ // Add new file to store files. Clear snapshot too while we have the Store write lock.
2372+ return updateStorefiles (storeFiles , snapshot .getId ());
23852373 }
23862374
23872375 @ Override
0 commit comments