HBASE-14234 Exception encountered in WALProcedureStore#rollWriter() should be properly handled
This commit is contained in:
parent
395ec5a9bb
commit
71d3d24d8b
|
@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
|
|||
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
@ -681,14 +682,23 @@ public class WALProcedureStore extends ProcedureStoreBase {
|
|||
FSDataOutputStream newStream = null;
|
||||
Path newLogFile = null;
|
||||
long startPos = -1;
|
||||
newLogFile = getLogFilePath(logId);
|
||||
try {
|
||||
newLogFile = getLogFilePath(logId);
|
||||
newStream = fs.create(newLogFile, false);
|
||||
ProcedureWALFormat.writeHeader(newStream, header);
|
||||
startPos = newStream.getPos();
|
||||
} catch (FileAlreadyExistsException e) {
|
||||
LOG.error("Log file with id=" + logId + " already exists", e);
|
||||
return false;
|
||||
} catch (RemoteException re) {
|
||||
LOG.warn("failed to create log file with id=" + logId, re);
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
ProcedureWALFormat.writeHeader(newStream, header);
|
||||
startPos = newStream.getPos();
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Encountered exception writing header", ioe);
|
||||
newStream.close();
|
||||
return false;
|
||||
}
|
||||
lock.lock();
|
||||
try {
|
||||
|
|
Loading…
Reference in New Issue