/** * 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; required uint64 segmentTxnId = 5; } 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 { } /** * purgeLogs() */ message PurgeLogsRequestProto { required RequestInfoProto reqInfo = 1; required uint64 minTxIdToKeep = 2; } message PurgeLogsResponseProto { } /** * getJournalState() */ message GetJournalStateRequestProto { required JournalIdProto jid = 1; } message GetJournalStateResponseProto { required uint64 lastPromisedEpoch = 1; required uint32 httpPort = 2; } /** * format() */ message FormatRequestProto { required JournalIdProto jid = 1; required NamespaceInfoProto nsInfo = 2; } message FormatResponseProto { } /** * 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; required uint64 lastWriterEpoch = 3; } /** * 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 format(FormatRequestProto) returns (FormatResponseProto); rpc journal(JournalRequestProto) returns (JournalResponseProto); rpc startLogSegment(StartLogSegmentRequestProto) returns (StartLogSegmentResponseProto); rpc finalizeLogSegment(FinalizeLogSegmentRequestProto) returns (FinalizeLogSegmentResponseProto); rpc purgeLogs(PurgeLogsRequestProto) returns (PurgeLogsResponseProto); rpc getEditLogManifest(GetEditLogManifestRequestProto) returns (GetEditLogManifestResponseProto); rpc prepareRecovery(PrepareRecoveryRequestProto) returns (PrepareRecoveryResponseProto); rpc acceptRecovery(AcceptRecoveryRequestProto) returns (AcceptRecoveryResponseProto); }