hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Tsz Wo Nicholas Sze (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDDS-638) enable ratis snapshots for HDDS datanodes
Date Sun, 21 Oct 2018 11:53:00 GMT

    [ https://issues.apache.org/jira/browse/HDDS-638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16658188#comment-16658188
] 

Tsz Wo Nicholas Sze commented on HDDS-638:
------------------------------------------

Some comments on ContainerStateMachine:
- For updating lastAppliedTermIndex:
-* ConcurrentHashMap does not support null values. It won't work since addRequest always returns
null.  Just remove addRequest.
-* Remove addRequest(..) since it do not seem useful.
-* lastSuccessfullyAppliedIndex does not seem useful. How about removing it? Just use lastAppliedTermIndex
in BaseStateMachine.

The code should look like below:
{code}
+  private void updateLastAppliedTermIndex() {
+    Long appledTerm = null;
+    long appliedIndex = -1;
+    for(long i = getLastAppliedTermIndex().getIndex() + 1;; i++) {
+      final Long removed = containerCommandCompletionMap.remove(i);
+      if (removed == null) {
+        break;
+      }
+      appledTerm = removed;
+      appliedIndex = i;
+    }
+    if (appledTerm != null) {
+      updateLastAppliedTermIndex(appliedIndex, appledTerm);
+    }
+  }
+
   /*
    * ApplyTransaction calls in Ratis are sequential.
    */
   @Override
   public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+    final long index = trx.getLogEntry().getIndex();
     try {
       metrics.incNumApplyTransactionsOps();
       ContainerCommandRequestProto requestProto =
@@ -418,7 +476,7 @@ private ByteString readStateMachineData(LogEntryProto entry,
               blockDataProto.getBlockID());
           return completeExceptionally(ioe);
         }
-        blockData.setBlockCommitSequenceId(trx.getLogEntry().getIndex());
+        blockData.setBlockCommitSequenceId(index);
         final ContainerProtos.PutBlockRequestProto putBlockRequestProto =
             ContainerProtos.PutBlockRequestProto
                 .newBuilder(requestProto.getPutBlock())
@@ -440,6 +498,13 @@ private ByteString readStateMachineData(LogEntryProto entry,
         future.thenApply(
             r -> createContainerFutureMap.remove(containerID).complete(null));
       }
+
+      future.thenAccept( m -> {
+        final Long previous = containerCommandCompletionMap.put(index, trx.getLogEntry().getTerm());
+        Preconditions.checkState(previous == null);
+        updateLastAppliedTermIndex();
+      });
+
       return future;
     } catch (IOException e) {
       metrics.incNumApplyTransactionsFails();
{code}


- Why "TODO persist open containers in snapshots"?  Open containers should be persisted if
the index is applied to state machine.  No?

- In loadSnapshot, remove the snapshotFile.exists() check.  It must exist by storage.getLatestSnapshot().
-* Remove the warning from the snapshot == null case.  It is normal when the storage is newly
formatted.

- Add @Override to takeSnapshot() and it should throw IOException when createNewFile() fails.

- In the test, it should check if the expected snapshot files exists.


Some other comments:

- flushStateMachineData is expensive since it loops through the entire map.  It should be
rewritten (probably in a sepearted JIRA.)

- The following TODO in initialize(.,) can be removed.  BaseStateMachine.getId() will the
server id iff initialize has been called; otherwise, it returns null.	
{code}
    // TODO: Add a flag that tells you that initialize has been called.
    // Check with Ratis if this feature is done in Ratis.
{code}

> enable ratis snapshots for HDDS datanodes
> -----------------------------------------
>
>                 Key: HDDS-638
>                 URL: https://issues.apache.org/jira/browse/HDDS-638
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>          Components: Ozone Datanode
>    Affects Versions: 0.3.0
>            Reporter: Mukul Kumar Singh
>            Assignee: Mukul Kumar Singh
>            Priority: Blocker
>         Attachments: HDDS-638.001.patch
>
>
> Currently on a restart, a hdds datanode, starts applying log entries from the start of
the log.
> This should can be avoided by taking a ratis snapshot to persist the last stable state
and on restart the datanodes start applying log from that index.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org


Mime
View raw message