Skip to content

Commit e2f2aeb

Browse files
authored
HDDS-11548. Add some logging to the StateMachine (#7291)
1 parent 523c860 commit e2f2aeb

File tree

4 files changed

+34
-12
lines changed

4 files changed

+34
-12
lines changed

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -284,6 +284,7 @@ public void initialize(
284284
storage.init(raftStorage);
285285
ratisServer.notifyGroupAdd(gid);
286286

287+
LOG.info("{}: initialize {}", server.getId(), id);
287288
loadSnapshot(storage.getLatestSnapshot());
288289
}
289290

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMHADBTransactionBufferImpl.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,8 @@
2626
import org.apache.hadoop.hdds.utils.db.BatchOperation;
2727
import org.apache.hadoop.hdds.utils.db.Table;
2828
import org.apache.ratis.statemachine.SnapshotInfo;
29+
import org.slf4j.Logger;
30+
import org.slf4j.LoggerFactory;
2931

3032
import java.io.IOException;
3133
import java.util.concurrent.atomic.AtomicLong;
@@ -41,6 +43,8 @@
4143
* operation in DB.
4244
*/
4345
public class SCMHADBTransactionBufferImpl implements SCMHADBTransactionBuffer {
46+
47+
public static final Logger LOG = LoggerFactory.getLogger(SCMHADBTransactionBufferImpl.class);
4448
private final StorageContainerManager scm;
4549
private SCMMetadataStore metadataStore;
4650
private BatchOperation currentBatchOperation;
@@ -107,6 +111,8 @@ public SnapshotInfo getLatestSnapshot() {
107111

108112
@Override
109113
public void setLatestSnapshot(SnapshotInfo latestSnapshot) {
114+
LOG.info("{}: Set latest Snapshot to {}",
115+
scm.getScmHAManager().getRatisServer().getDivision().getId(), latestSnapshot);
110116
this.latestSnapshot.set(latestSnapshot);
111117
}
112118

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMStateMachine.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -137,6 +137,7 @@ public void initialize(RaftServer server, RaftGroupId id,
137137
getLifeCycle().startAndTransition(() -> {
138138
super.initialize(server, id, raftStorage);
139139
storage.init(raftStorage);
140+
LOG.info("{}: initialize {}", server.getId(), id);
140141
});
141142
}
142143

@@ -149,6 +150,9 @@ public CompletableFuture<Message> applyTransaction(
149150
final SCMRatisRequest request = SCMRatisRequest.decode(
150151
Message.valueOf(trx.getStateMachineLogEntry().getLogData()));
151152

153+
if (LOG.isDebugEnabled()) {
154+
LOG.debug("{}: applyTransaction {}", getId(), TermIndex.valueOf(trx.getLogEntry()));
155+
}
152156
try {
153157
applyTransactionFuture.complete(process(request));
154158
} catch (SCMException ex) {
@@ -389,6 +393,7 @@ public void notifyConfigurationChanged(long term, long index,
389393
@Override
390394
public void pause() {
391395
final LifeCycle lc = getLifeCycle();
396+
LOG.info("{}: Try to pause from current LifeCycle state {}", getId(), lc);
392397
if (lc.getCurrentState() != LifeCycle.State.NEW) {
393398
lc.transition(LifeCycle.State.PAUSING);
394399
lc.transition(LifeCycle.State.PAUSED);
@@ -414,6 +419,8 @@ public void reinitialize() throws IOException {
414419
throw new IOException(e);
415420
}
416421

422+
LOG.info("{}: SCMStateMachine is reinitializing. newTermIndex = {}", getId(), termIndex);
423+
417424
// re-initialize the DBTransactionBuffer and update the lastAppliedIndex.
418425
try {
419426
transactionBuffer.init();

hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java

Lines changed: 20 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,7 @@
4646
import org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler;
4747
import org.apache.hadoop.ozone.protocolPB.RequestHandler;
4848
import org.apache.hadoop.security.UserGroupInformation;
49+
import org.apache.hadoop.util.Time;
4950
import org.apache.hadoop.util.concurrent.HadoopExecutors;
5051
import org.apache.ratis.proto.RaftProtos;
5152
import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
@@ -88,7 +89,6 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
8889
new SimpleStateMachineStorage();
8990
private final OzoneManager ozoneManager;
9091
private RequestHandler handler;
91-
private RaftGroupId raftGroupId;
9292
private volatile OzoneManagerDoubleBuffer ozoneManagerDoubleBuffer;
9393
private final ExecutorService executorService;
9494
private final ExecutorService installSnapshotExecutor;
@@ -134,17 +134,18 @@ public void initialize(RaftServer server, RaftGroupId id,
134134
RaftStorage raftStorage) throws IOException {
135135
getLifeCycle().startAndTransition(() -> {
136136
super.initialize(server, id, raftStorage);
137-
this.raftGroupId = id;
138137
storage.init(raftStorage);
138+
LOG.info("{}: initialize {} with {}", getId(), id, getLastAppliedTermIndex());
139139
});
140140
}
141141

142142
@Override
143143
public synchronized void reinitialize() throws IOException {
144144
loadSnapshotInfoFromDB();
145145
if (getLifeCycleState() == LifeCycle.State.PAUSED) {
146-
unpause(getLastAppliedTermIndex().getIndex(),
147-
getLastAppliedTermIndex().getTerm());
146+
final TermIndex lastApplied = getLastAppliedTermIndex();
147+
unpause(lastApplied.getIndex(), lastApplied.getTerm());
148+
LOG.info("{}: reinitialize {} with {}", getId(), getGroupId(), lastApplied);
148149
}
149150
}
150151

@@ -160,6 +161,7 @@ public void notifyLeaderChanged(RaftGroupMemberId groupMemberId,
160161
RaftPeerId newLeaderId) {
161162
// Initialize OMHAMetrics
162163
ozoneManager.omHAMetricsInit(newLeaderId.toString());
164+
LOG.info("{}: leader changed to {}", groupMemberId, newLeaderId);
163165
}
164166

165167
/** Notified by Ratis for non-StateMachine term-index update. */
@@ -263,7 +265,7 @@ public TransactionContext startTransaction(
263265
messageContent);
264266

265267
Preconditions.checkArgument(raftClientRequest.getRaftGroupId().equals(
266-
raftGroupId));
268+
getGroupId()));
267269
try {
268270
handler.validateRequest(omRequest);
269271
} catch (IOException ioe) {
@@ -293,6 +295,10 @@ public TransactionContext preAppendTransaction(TransactionContext trx)
293295

294296
OzoneManagerPrepareState prepareState = ozoneManager.getPrepareState();
295297

298+
if (LOG.isDebugEnabled()) {
299+
LOG.debug("{}: preAppendTransaction {}", getId(), TermIndex.valueOf(trx.getLogEntry()));
300+
}
301+
296302
if (cmdType == OzoneManagerProtocolProtos.Type.Prepare) {
297303
// Must authenticate prepare requests here, since we must determine
298304
// whether or not to apply the prepare gate before proceeding with the
@@ -303,8 +309,7 @@ public TransactionContext preAppendTransaction(TransactionContext trx)
303309
if (ozoneManager.getAclsEnabled()
304310
&& !ozoneManager.isAdmin(userGroupInformation)) {
305311
String message = "Access denied for user " + userGroupInformation
306-
+ ". "
307-
+ "Superuser privilege is required to prepare ozone managers.";
312+
+ ". Superuser privilege is required to prepare upgrade/downgrade.";
308313
OMException cause =
309314
new OMException(message, OMException.ResultCodes.ACCESS_DENIED);
310315
// Leader should not step down because of this failure.
@@ -341,6 +346,7 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
341346
: OMRatisHelper.convertByteStringToOMRequest(
342347
trx.getStateMachineLogEntry().getLogData());
343348
final TermIndex termIndex = TermIndex.valueOf(trx.getLogEntry());
349+
LOG.debug("{}: applyTransaction {}", getId(), termIndex);
344350
// In the current approach we have one single global thread executor.
345351
// with single thread. Right now this is being done for correctness, as
346352
// applyTransaction will be run on multiple OM's we want to execute the
@@ -427,12 +433,14 @@ public synchronized void pause() {
427433
*/
428434
public synchronized void unpause(long newLastAppliedSnaphsotIndex,
429435
long newLastAppliedSnapShotTermIndex) {
430-
LOG.info("OzoneManagerStateMachine is un-pausing");
431436
if (statePausedCount.decrementAndGet() == 0) {
432437
getLifeCycle().startAndTransition(() -> {
433438
this.ozoneManagerDoubleBuffer = buildDoubleBufferForRatis();
434439
this.setLastAppliedTermIndex(TermIndex.valueOf(
435440
newLastAppliedSnapShotTermIndex, newLastAppliedSnaphsotIndex));
441+
LOG.info("{}: OzoneManagerStateMachine un-pause completed. " +
442+
"newLastAppliedSnaphsotIndex: {}, newLastAppliedSnapShotTermIndex: {}",
443+
getId(), newLastAppliedSnaphsotIndex, newLastAppliedSnapShotTermIndex);
436444
});
437445
}
438446
}
@@ -482,15 +490,15 @@ private synchronized long takeSnapshotImpl() throws IOException {
482490
final TermIndex applied = getLastAppliedTermIndex();
483491
final TermIndex notified = getLastNotifiedTermIndex();
484492
final TermIndex snapshot = applied.compareTo(notified) > 0 ? applied : notified;
485-
LOG.info(" applied = {}", applied);
486-
LOG.info(" skipped = {}", lastSkippedIndex);
487-
LOG.info("notified = {}", notified);
488-
LOG.info("snapshot = {}", snapshot);
489493

494+
long startTime = Time.monotonicNow();
490495
final TransactionInfo transactionInfo = TransactionInfo.valueOf(snapshot);
491496
ozoneManager.setTransactionInfo(transactionInfo);
492497
ozoneManager.getMetadataManager().getTransactionInfoTable().put(TRANSACTION_INFO_KEY, transactionInfo);
493498
ozoneManager.getMetadataManager().getStore().flushDB();
499+
LOG.info("{}: taking snapshot. applied = {}, skipped = {}, " +
500+
"notified = {}, current snapshot index = {}, took {} ms",
501+
getId(), applied, lastSkippedIndex, notified, snapshot, Time.monotonicNow() - startTime);
494502
return snapshot.getIndex();
495503
}
496504

0 commit comments

Comments
 (0)