HBASE-12614 Potentially unclosed StoreFile(s) in DefaultCompactor#compact()
This commit is contained in:
parent
23382fe625
commit
b189b25ab3
|
@ -24,6 +24,8 @@ import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -38,6 +40,8 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class DefaultCompactor extends Compactor {
|
public class DefaultCompactor extends Compactor {
|
||||||
|
private static final Log LOG = LogFactory.getLog(DefaultCompactor.class);
|
||||||
|
|
||||||
public DefaultCompactor(final Configuration conf, final Store store) {
|
public DefaultCompactor(final Configuration conf, final Store store) {
|
||||||
super(conf, store);
|
super(conf, store);
|
||||||
}
|
}
|
||||||
|
@ -127,7 +131,11 @@ public class DefaultCompactor extends Compactor {
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
for (StoreFile f : readersToClose) {
|
for (StoreFile f : readersToClose) {
|
||||||
f.closeReader(true);
|
try {
|
||||||
|
f.closeReader(true);
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.warn("Exception closing " + f, ioe);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue