|
@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PersistedRecoveryPaxosData;
|
|
@@ -114,6 +115,8 @@ class Journal implements Closeable {
|
|
|
Preconditions.checkState(nsInfo.getNamespaceID() != 0,
|
|
|
"can't format with uninitialized namespace info: %s",
|
|
|
nsInfo);
|
|
|
+ LOG.info("Formatting " + this + " with namespace info: " +
|
|
|
+ nsInfo);
|
|
|
storage.format(nsInfo);
|
|
|
}
|
|
|
|
|
@@ -134,6 +137,7 @@ class Journal implements Closeable {
|
|
|
* any lower epoch, or 0 if no promises have been made.
|
|
|
*/
|
|
|
synchronized long getLastPromisedEpoch() throws IOException {
|
|
|
+ checkFormatted();
|
|
|
return lastPromisedEpoch.get();
|
|
|
}
|
|
|
|
|
@@ -150,9 +154,8 @@ class Journal implements Closeable {
|
|
|
synchronized NewEpochResponseProto newEpoch(
|
|
|
NamespaceInfo nsInfo, long epoch) throws IOException {
|
|
|
|
|
|
- // If the storage is unformatted, format it with this NS.
|
|
|
- // Otherwise, check that the NN's nsinfo matches the storage.
|
|
|
- storage.formatIfNecessary(nsInfo);
|
|
|
+ checkFormatted();
|
|
|
+ storage.checkConsistentNamespace(nsInfo);
|
|
|
|
|
|
if (epoch <= getLastPromisedEpoch()) {
|
|
|
throw new IOException("Proposed epoch " + epoch + " <= last promise " +
|
|
@@ -185,6 +188,7 @@ class Journal implements Closeable {
|
|
|
synchronized void journal(RequestInfo reqInfo, long firstTxnId,
|
|
|
int numTxns, byte[] records) throws IOException {
|
|
|
checkRequest(reqInfo);
|
|
|
+ checkFormatted();
|
|
|
|
|
|
// TODO: if a JN goes down and comes back up, then it will throw
|
|
|
// this exception on every edit. We should instead send back
|
|
@@ -226,6 +230,13 @@ class Journal implements Closeable {
|
|
|
// TODO: some check on serial number that they only increase from a given
|
|
|
// client
|
|
|
}
|
|
|
+
|
|
|
+ private void checkFormatted() throws JournalNotFormattedException {
|
|
|
+ if (!storage.isFormatted()) {
|
|
|
+ throw new JournalNotFormattedException("Journal " + storage +
|
|
|
+ " not formatted");
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Start a new segment at the given txid. The previous segment
|
|
@@ -235,6 +246,7 @@ class Journal implements Closeable {
|
|
|
throws IOException {
|
|
|
assert fjm != null;
|
|
|
checkRequest(reqInfo);
|
|
|
+ checkFormatted();
|
|
|
|
|
|
Preconditions.checkState(curSegment == null,
|
|
|
"Can't start a log segment, already writing " + curSegment);
|
|
@@ -251,6 +263,7 @@ class Journal implements Closeable {
|
|
|
public synchronized void finalizeLogSegment(RequestInfo reqInfo, long startTxId,
|
|
|
long endTxId) throws IOException {
|
|
|
checkRequest(reqInfo);
|
|
|
+ checkFormatted();
|
|
|
|
|
|
if (startTxId == curSegmentTxId) {
|
|
|
if (curSegment != null) {
|
|
@@ -284,6 +297,7 @@ class Journal implements Closeable {
|
|
|
public synchronized void purgeLogsOlderThan(RequestInfo reqInfo,
|
|
|
long minTxIdToKeep) throws IOException {
|
|
|
checkRequest(reqInfo);
|
|
|
+ checkFormatted();
|
|
|
|
|
|
fjm.purgeLogsOlderThan(minTxIdToKeep);
|
|
|
purgePaxosDecisionsOlderThan(minTxIdToKeep);
|
|
@@ -320,6 +334,8 @@ class Journal implements Closeable {
|
|
|
throws IOException {
|
|
|
// No need to checkRequest() here - anyone may ask for the list
|
|
|
// of segments.
|
|
|
+ checkFormatted();
|
|
|
+
|
|
|
RemoteEditLogManifest manifest = new RemoteEditLogManifest(
|
|
|
fjm.getRemoteEditLogs(sinceTxId));
|
|
|
return manifest;
|
|
@@ -360,6 +376,7 @@ class Journal implements Closeable {
|
|
|
public synchronized PrepareRecoveryResponseProto prepareRecovery(
|
|
|
RequestInfo reqInfo, long segmentTxId) throws IOException {
|
|
|
checkRequest(reqInfo);
|
|
|
+ checkFormatted();
|
|
|
|
|
|
PrepareRecoveryResponseProto.Builder builder =
|
|
|
PrepareRecoveryResponseProto.newBuilder();
|
|
@@ -388,6 +405,7 @@ class Journal implements Closeable {
|
|
|
SegmentStateProto segment, URL fromUrl)
|
|
|
throws IOException {
|
|
|
checkRequest(reqInfo);
|
|
|
+ checkFormatted();
|
|
|
long segmentTxId = segment.getStartTxId();
|
|
|
|
|
|
// TODO: right now, a recovery of a segment when the log is
|