HBASE-11815 Flush and compaction could just close the tmp writer if there

is an exception
This commit is contained in:
Ramkrishna 2014-10-07 09:40:34 +05:30
parent baaaac0202
commit eb6f196395
2 changed files with 25 additions and 6 deletions

View File

@ -66,10 +66,19 @@ public class DefaultStoreFlusher extends StoreFlusher {
writer = store.createWriterInTmp(
cellsCount, store.getFamily().getCompression(), false, true, true);
writer.setTimeRangeTracker(snapshot.getTimeRangeTracker());
IOException e = null;
try {
performFlush(scanner, writer, smallestReadPoint);
} catch (IOException ioe) {
e = ioe;
// throw the exception out
throw ioe;
} finally {
finalizeWriter(writer, cacheFlushId, status);
if (e != null) {
writer.close();
} else {
finalizeWriter(writer, cacheFlushId, status);
}
}
}
} finally {

View File

@ -26,9 +26,9 @@ import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
@ -55,6 +55,7 @@ public class DefaultCompactor extends Compactor {
StoreFile.Writer writer = null;
List<Path> newFiles = new ArrayList<Path>();
boolean cleanSeqId = false;
IOException e = null;
try {
InternalScanner scanner = null;
try {
@ -92,11 +93,20 @@ public class DefaultCompactor extends Compactor {
scanner.close();
}
}
} finally {
} catch (IOException ioe) {
e = ioe;
// Throw the exception
throw ioe;
}
finally {
if (writer != null) {
writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
writer.close();
newFiles.add(writer.getPath());
if (e != null) {
writer.close();
} else {
writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
writer.close();
newFiles.add(writer.getPath());
}
}
}
return newFiles;