Repository: hbase Updated Branches: refs/heads/branch-1 226873f1a -> de15b1fd9
HBASE-11815 - Flush and compaction could just close the tmp writer if there is an exception Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/de15b1fd Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/de15b1fd Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/de15b1fd Branch: refs/heads/branch-1 Commit: de15b1fd98da9f960a782c4dd2c0cd1dd852cf7c Parents: 226873f Author: Ramkrishna <[email protected]> Authored: Tue Oct 7 09:42:21 2014 +0530 Committer: Ramkrishna <[email protected]> Committed: Tue Oct 7 09:42:21 2014 +0530 ---------------------------------------------------------------------- .../hbase/regionserver/DefaultStoreFlusher.java | 11 ++++++++++- .../compactions/DefaultCompactor.java | 20 +++++++++++++++----- 2 files changed, 25 insertions(+), 6 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/de15b1fd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java index bcf5e22..73b8cb9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java @@ -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 { http://git-wip-us.apache.org/repos/asf/hbase/blob/de15b1fd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java index 7139ea8..aae3968 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java @@ -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;
