[ 
https://issues.apache.org/jira/browse/HDDS-1512?focusedWorklogId=243791&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-243791
 ]

ASF GitHub Bot logged work on HDDS-1512:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 17/May/19 02:21
            Start Date: 17/May/19 02:21
    Worklog Time Spent: 10m 
      Work Description: linyiqun commented on pull request #810: HDDS-1512. 
Implement DoubleBuffer in OzoneManager.
URL: https://github.com/apache/hadoop/pull/810#discussion_r284957277
 
 

 ##########
 File path: 
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java
 ##########
 @@ -0,0 +1,212 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+
+package org.apache.hadoop.ozone.om.ratis;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.ratis.helpers.DoubleBufferEntry;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A double-buffer for OM requests.
+ */
+public class OzoneManagerDoubleBuffer {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneManagerDoubleBuffer.class.getName());
+
+  private TransactionBuffer currentBuffer;
+  private TransactionBuffer readyBuffer;
+  private Daemon daemon;
+  private volatile boolean syncToDB;
+  private final OMMetadataManager omMetadataManager;
+  private AtomicLong flushedTransactionCount = new AtomicLong(0);
+  private AtomicLong flushIterations = new AtomicLong(0);
+
+  public OzoneManagerDoubleBuffer(OMMetadataManager omMetadataManager) {
+    this.currentBuffer = new TransactionBuffer();
+    this.readyBuffer = new TransactionBuffer();
+    this.omMetadataManager = omMetadataManager;
+
+    // Daemon thread which runs in back ground and flushes transactions to DB.
+    daemon = new Daemon(this::flushTransactions);
+    daemon.start();
 
 Review comment:
   Actually I  mean not to start flushTransactions thread in OMDoubleBuffer. 
It's would  be  better to let the caller of function flushTransactions outside 
of the class OMDoubleBuffer.  How the flush transactions behavior should be 
triggered by outside not by itself I think.
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 243791)
    Time Spent: 2h 10m  (was: 2h)

> Implement DoubleBuffer in OzoneManager
> --------------------------------------
>
>                 Key: HDDS-1512
>                 URL: https://issues.apache.org/jira/browse/HDDS-1512
>             Project: Hadoop Distributed Data Store
>          Issue Type: Sub-task
>          Components: Ozone Manager
>            Reporter: Bharat Viswanadham
>            Assignee: Bharat Viswanadham
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> This Jira is created to implement DoubleBuffer in OzoneManager to flush 
> transactions to OM DB.
>  
> h2. Flushing Transactions to RocksDB:
> We propose using an implementation similar to the HDFS EditsDoubleBuffer.  We 
> shall flush RocksDB transactions in batches, instead of current way of using 
> rocksdb.put() after every operation. At a given time only one batch will be 
> outstanding for flush while newer transactions are accumulated in memory to 
> be flushed later.
>  
> In DoubleBuffer it will have 2 buffers one is currentBuffer, and the other is 
> readyBuffer. We add entry to current buffer, and we check if another flush 
> call is outstanding. If not, we flush to disk Otherwise we add entries to 
> otherBuffer while sync is happening.
>  
> In this if sync is happening, we shall add new requests to other buffer and 
> when we can sync we use *RocksDB batch commit to sync to disk, instead of 
> rocksdb put.*
>  
> Note: If flush to disk is failed on any OM, we shall terminate the 
> OzoneManager, so that OM DB’s will not diverge. Flush failure should be 
> considered as catastrophic failure.
>  
> Scope of this Jira is to add DoubleBuffer implementation, integrating to 
> current OM will be done in further jira's.
>  



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

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

Reply via email to