You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "adoroszlai (via GitHub)" <gi...@apache.org> on 2023/06/09 13:04:21 UTC

[GitHub] [ozone] adoroszlai opened a new pull request, #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

adoroszlai opened a new pull request, #4866:
URL: https://github.com/apache/ozone/pull/4866

   ## What changes were proposed in this pull request?
   
   `TestScmHAFinalization` intermittently reports `Found 2 leaked objects` (`CodecBuffer` instances).  `FinalizationStateManagerImpl` is leaking a pair of `CodecBuffer`s allocated for writing layout version to SCM metadata store:
   
   ```
   2023-06-08 19:50:12,664 [Finalizer] WARN  db.CodecBuffer (CodecBuffer.java:finalize(129)) - LEAK 1: org.apache.hadoop.hdds.utils.db.CodecBuffer@292f4cc4, refCnt=1, capacity=3 allocation:
   org.apache.hadoop.hdds.utils.db.CodecBuffer.allocate(CodecBuffer.java:74)
   ...
   org.apache.hadoop.hdds.utils.db.TypedTable.putWithBatch(TypedTable.java:172)
   org.apache.hadoop.hdds.scm.ha.SCMHADBTransactionBufferImpl.addToBuffer(SCMHADBTransactionBufferImpl.java:70)
   org.apache.hadoop.hdds.scm.server.upgrade.FinalizationStateManagerImpl.finalizeLayoutFeatureLocal(FinalizationStateManagerImpl.java:156)
   org.apache.hadoop.hdds.scm.server.upgrade.FinalizationStateManagerImpl.reinitialize(FinalizationStateManagerImpl.java:257)
   ...
   org.apache.hadoop.hdds.scm.server.upgrade.FinalizationManagerImpl.reinitialize(FinalizationManagerImpl.java:147)
   org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.startServices(SCMHAManagerImpl.java:445)
   org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.reloadSCMState(SCMHAManagerImpl.java:357)
   org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.installCheckpoint(SCMHAManagerImpl.java:308)
   org.apache.hadoop.hdds.scm.ha.SCMHAManagerImpl.installCheckpoint(SCMHAManagerImpl.java:258)
   org.apache.hadoop.hdds.scm.ha.SCMStateMachine.reinitialize(SCMStateMachine.java:391)
   org.apache.ratis.server.impl.StateMachineUpdater.reload(StateMachineUpdater.java:218)
   ```
   
   `finalizeLayoutFeatureLocal` adds these to the `transactionBuffer`, collecting them in a batch operation:
   
   https://github.com/apache/ozone/blob/b479a384e854b6eb62c429801ec3ce2acfa3c160/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManagerImpl.java#L156-L157
   
   `CodecBuffer`s are released on commit, which happens when transaction buffer is flushed.
   
   https://github.com/apache/ozone/blob/b479a384e854b6eb62c429801ec3ce2acfa3c160/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBBatchOperation.java#L333-L339
   
   The problem is that the buffer may be closed without being flushed.  In this case the contents of the current batch, if any, are not committed.  (The batch is closed, so RocksDB objects are not leaked.)
   
   We could fix it by explicitly flushing the buffer when finalization completes.  However, the problem may be more generic, any metadata stored by SCM via the transaction buffer may be lost.  So this PR proposes to commit the in-progress batch operation, if any, before closing it.
   
   Also:
    * Add a `closed` flag.  Allow closing only once.  Reject operations if closed.
    * Add precondition for `currentBatchOperation` being non-null.
    * Replace `AtomicLong txFlushPending` with a `boolean`, since the specific number is not important, only `> 0` or `== 0` cases are distinguished.
    * Change `SCMMetadataStore` to `final` both in `SCMHADBTransactionBufferImpl` and in `StorageContainerManager`
   
   https://issues.apache.org/jira/browse/HDDS-8740
   
   ## How was this patch tested?
   
   `TestScmHAFinalization#testSnapshotFinalization` passed in 30x10 runs:
   https://github.com/adoroszlai/hadoop-ozone/actions/runs/5221131625
   
   Regular CI:
   https://github.com/adoroszlai/hadoop-ozone/actions/runs/5221128846


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] adoroszlai commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1585568032

   > how about we clear the cache in `close`? Then, it will also work for OM, datanodes and other cases.
   
   Thanks @szetszwo for the suggestion.  After posting this PR I was thinking along the same lines, but haven't updated the PR yet.
   
   https://github.com/adoroszlai/hadoop-ozone/commit/7a4448c5e306bbc30e5576535bf46f138a748fef


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] szetszwo commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "szetszwo (via GitHub)" <gi...@apache.org>.
szetszwo commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1586050134

   > Can we assume that all batches should be committed (to avoid data loss)? ...
   
   We should support `close` without `commit`.  It is like `abort`.  It is useful when a sequence of operations failing in the middle (e.g. Atomically put 10 records but it fails to read the 8th record from another source.) Then, it should abort instead of committing partially.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] adoroszlai closed pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai closed pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()
URL: https://github.com/apache/ozone/pull/4866


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] adoroszlai commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1590446012

   > should not be a problem since the information would be replayed from the Ratis log on restart
   
   Then fixing the leak itself in HDDS-8803 seems enough.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] errose28 commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "errose28 (via GitHub)" <gi...@apache.org>.
errose28 commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1590156169

   In what cases would the buffer be closed before flush other than shutdown? Shutdown should not be a problem since the information would be replayed from the Ratis log on restart. If the buffer is closed and entries are not flushed but the SCM keeps accepting transactions, that seems like a larger problem outside of finalization.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] szetszwo commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "szetszwo (via GitHub)" <gi...@apache.org>.
szetszwo commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1587739303

   @adoroszlai , after HDDS-8803, do we still need this?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] szetszwo commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "szetszwo (via GitHub)" <gi...@apache.org>.
szetszwo commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1585538596

   @adoroszlai , how about we clear the cache in `close`?   Then, it will also work for OM, datanodes and other cases.
   ```java
   +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBBatchOperation.java
   @@ -350,6 +350,7 @@ public void commit(RocksDatabase db, ManagedWriteOptions writeOptions)
      @Override
      public void close() {
        debug(() -> String.format("%s: close", name));
   +    opCache.clear();
        writeBatch.close();
      }
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [ozone] adoroszlai commented on pull request #4866: HDDS-8740. Commit batch in SCMHADBTransactionBufferImpl on close()

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4866:
URL: https://github.com/apache/ozone/pull/4866#issuecomment-1587860686

   > after [HDDS-8803](https://issues.apache.org/jira/browse/HDDS-8803), do we still need this?
   
   @szetszwo This is no longer necessary for the leak.  It would be nice if @errose28 could check if losing the layout version due to uncommitted batch (buffer not flushed) may be a problem.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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