2012-07-20 00:25:50 +00:00
|
|
|
/**
|
|
|
|
* Licensed to the Apache Software Foundation (ASF) under one
|
|
|
|
* or more contributor license agreements. See the NOTICE file
|
|
|
|
* distributed with this work for additional information
|
|
|
|
* regarding copyright ownership. The ASF licenses this file
|
|
|
|
* to you under the Apache License, Version 2.0 (the
|
|
|
|
* "License"); you may not use this file except in compliance
|
|
|
|
* with the License. You may obtain a copy of the License at
|
|
|
|
*
|
|
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
*
|
|
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
|
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
* See the License for the specific language governing permissions and
|
|
|
|
* limitations under the License.
|
|
|
|
*/
|
|
|
|
|
|
|
|
option java_package = "org.apache.hadoop.hdfs.qjournal.protocol";
|
|
|
|
option java_outer_classname = "QJournalProtocolProtos";
|
|
|
|
option java_generic_services = true;
|
|
|
|
option java_generate_equals_and_hash = true;
|
|
|
|
|
|
|
|
import "hdfs.proto";
|
|
|
|
|
|
|
|
message JournalIdProto {
|
|
|
|
required string identifier = 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
message RequestInfoProto {
|
|
|
|
required JournalIdProto journalId = 1;
|
|
|
|
required uint64 epoch = 2;
|
|
|
|
required uint64 ipcSerialNumber = 3;
|
|
|
|
}
|
|
|
|
|
|
|
|
message SegmentStateProto {
|
|
|
|
required uint64 startTxId = 1;
|
|
|
|
required uint64 endTxId = 2;
|
|
|
|
required bool isInProgress = 3;
|
|
|
|
required bytes md5sum = 4;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* The storage format used on local disk for previously
|
|
|
|
* accepted decisions.
|
|
|
|
*/
|
|
|
|
message PersistedRecoveryPaxosData {
|
|
|
|
required SegmentStateProto segmentState = 1;
|
|
|
|
required uint64 acceptedInEpoch = 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* journal()
|
|
|
|
*/
|
|
|
|
|
|
|
|
message JournalRequestProto {
|
|
|
|
required RequestInfoProto reqInfo = 1;
|
|
|
|
required uint64 firstTxnId = 2;
|
|
|
|
required uint32 numTxns = 3;
|
|
|
|
required bytes records = 4;
|
|
|
|
}
|
|
|
|
|
|
|
|
message JournalResponseProto {
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* startLogSegment()
|
|
|
|
*/
|
|
|
|
message StartLogSegmentRequestProto {
|
|
|
|
required RequestInfoProto reqInfo = 1;
|
|
|
|
required uint64 txid = 2; // Transaction ID
|
|
|
|
}
|
|
|
|
|
|
|
|
message StartLogSegmentResponseProto {
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* finalizeLogSegment()
|
|
|
|
*/
|
|
|
|
message FinalizeLogSegmentRequestProto {
|
|
|
|
required RequestInfoProto reqInfo = 1;
|
|
|
|
required uint64 startTxId = 2;
|
|
|
|
required uint64 endTxId = 3;
|
|
|
|
}
|
|
|
|
|
|
|
|
message FinalizeLogSegmentResponseProto {
|
|
|
|
}
|
|
|
|
|
2012-07-25 21:44:26 +00:00
|
|
|
/**
|
|
|
|
* purgeLogs()
|
|
|
|
*/
|
|
|
|
message PurgeLogsRequestProto {
|
|
|
|
required RequestInfoProto reqInfo = 1;
|
|
|
|
required uint64 minTxIdToKeep = 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
message PurgeLogsResponseProto {
|
|
|
|
}
|
|
|
|
|
2012-07-20 00:25:50 +00:00
|
|
|
/**
|
|
|
|
* getJournalState()
|
|
|
|
*/
|
|
|
|
message GetJournalStateRequestProto {
|
|
|
|
required JournalIdProto jid = 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
message GetJournalStateResponseProto {
|
|
|
|
required uint64 lastPromisedEpoch = 1;
|
|
|
|
required uint32 httpPort = 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* newEpoch()
|
|
|
|
*/
|
|
|
|
message NewEpochRequestProto {
|
|
|
|
required JournalIdProto jid = 1;
|
|
|
|
required NamespaceInfoProto nsInfo = 2;
|
|
|
|
required uint64 epoch = 3;
|
|
|
|
}
|
|
|
|
|
|
|
|
message NewEpochResponseProto {
|
|
|
|
optional uint64 lastSegmentTxId = 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* getEditLogManifest()
|
|
|
|
*/
|
|
|
|
message GetEditLogManifestRequestProto {
|
|
|
|
required JournalIdProto jid = 1;
|
|
|
|
required uint64 sinceTxId = 2; // Transaction ID
|
|
|
|
}
|
|
|
|
|
|
|
|
message GetEditLogManifestResponseProto {
|
|
|
|
required RemoteEditLogManifestProto manifest = 1;
|
|
|
|
required uint32 httpPort = 2;
|
|
|
|
|
|
|
|
// TODO: we should add nsinfo somewhere
|
|
|
|
// to verify that it matches up with our expectation
|
|
|
|
// required NamespaceInfoProto nsInfo = 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* prepareRecovery()
|
|
|
|
*/
|
|
|
|
message PrepareRecoveryRequestProto {
|
|
|
|
required RequestInfoProto reqInfo = 1;
|
|
|
|
required uint64 segmentTxId = 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
message PrepareRecoveryResponseProto {
|
|
|
|
optional SegmentStateProto segmentState = 1;
|
|
|
|
optional uint64 acceptedInEpoch = 2;
|
|
|
|
}
|
|
|
|
|
|
|
|
/**
|
|
|
|
* acceptRecovery()
|
|
|
|
*/
|
|
|
|
message AcceptRecoveryRequestProto {
|
|
|
|
required RequestInfoProto reqInfo = 1;
|
|
|
|
|
|
|
|
/** Details on the segment to recover */
|
|
|
|
required SegmentStateProto stateToAccept = 2;
|
|
|
|
|
|
|
|
/** The URL from which the log may be copied */
|
|
|
|
required string fromURL = 3;
|
|
|
|
}
|
|
|
|
|
|
|
|
message AcceptRecoveryResponseProto {
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
* Protocol used to journal edits to a JournalNode.
|
|
|
|
* See the request and response for details of rpc call.
|
|
|
|
*/
|
|
|
|
service QJournalProtocolService {
|
|
|
|
rpc getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto);
|
|
|
|
|
|
|
|
rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto);
|
|
|
|
|
|
|
|
rpc journal(JournalRequestProto) returns (JournalResponseProto);
|
|
|
|
|
|
|
|
rpc startLogSegment(StartLogSegmentRequestProto)
|
|
|
|
returns (StartLogSegmentResponseProto);
|
|
|
|
|
|
|
|
rpc finalizeLogSegment(FinalizeLogSegmentRequestProto)
|
|
|
|
returns (FinalizeLogSegmentResponseProto);
|
|
|
|
|
2012-07-25 21:44:26 +00:00
|
|
|
rpc purgeLogs(PurgeLogsRequestProto)
|
|
|
|
returns (PurgeLogsResponseProto);
|
|
|
|
|
2012-07-20 00:25:50 +00:00
|
|
|
rpc getEditLogManifest(GetEditLogManifestRequestProto)
|
|
|
|
returns (GetEditLogManifestResponseProto);
|
|
|
|
|
|
|
|
rpc prepareRecovery(PrepareRecoveryRequestProto)
|
|
|
|
returns (PrepareRecoveryResponseProto);
|
|
|
|
|
|
|
|
rpc acceptRecovery(AcceptRecoveryRequestProto)
|
|
|
|
returns (AcceptRecoveryResponseProto);
|
|
|
|
}
|