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

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

                Author: ASF GitHub Bot
            Created on: 08/May/19 23:54
            Start Date: 08/May/19 23:54
    Worklog Time Spent: 10m 
      Work Description: hanishakoneru commented on pull request #798: 
HDDS-1499. OzoneManager Cache.
URL: https://github.com/apache/hadoop/pull/798#discussion_r282297165
 
 

 ##########
 File path: 
hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/cache/PartialTableCache.java
 ##########
 @@ -0,0 +1,96 @@
+/*
+ * 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.
+ *
+ */
+
+package org.apache.hadoop.utils.db.cache;
+
+import java.util.Iterator;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+
+
+/**
+ * This is used for the tables where we don't want to cache entire table in
+ * in-memory.
+ */
+@Private
+@Evolving
+public class PartialTableCache<CACHEKEY, CACHEVALUE>
+    implements TableCache<CACHEKEY, CACHEVALUE>{
+
+  private final ConcurrentHashMap<CACHEKEY, CACHEVALUE> cache;
+  private final TreeSet<EpochEntry<CACHEKEY>> epochEntries;
+  private ExecutorService executorService;
+
+
+
+  public PartialTableCache() {
+    cache = new ConcurrentHashMap<>();
+    epochEntries = new TreeSet<EpochEntry<CACHEKEY>>();
+    // Created a singleThreadExecutor, so one cleanup will be running at a
+    // time.
+    executorService = Executors.newSingleThreadExecutor();
+  }
+
+  @Override
+  public CACHEVALUE get(CACHEKEY cachekey) {
+    return cache.get(cachekey);
+  }
+
+  @Override
+  public void put(CACHEKEY cacheKey, CACHEVALUE value) {
+    cache.put(cacheKey, value);
+    CacheValue cacheValue = (CacheValue) cache.get(cacheKey);
 
 Review comment:
   Instead of casting the cache.get() object to CacheValue, I think CACHEVALUE 
itself should extend CacheValue so that it is guaranteed that the Value part of 
TableCache is an instance of CacheValue.class. Same for CACHEKEY also.
 
----------------------------------------------------------------
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: 239520)
    Time Spent: 2h 10m  (was: 2h)

> OzoneManager Cache
> ------------------
>
>                 Key: HDDS-1499
>                 URL: https://issues.apache.org/jira/browse/HDDS-1499
>             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
>
> In this Jira, we shall implement a cache for Table.
> As with OM HA, we are planning to implement double buffer implementation to 
> flush transaction in a batch, instead of using rocksdb put() for every 
> operation. When this comes in to place we need cache in OzoneManager HA to 
> handle/server the requests for validation/returning responses.
>  
> This Jira will implement Cache as an integral part of the table. In this way 
> users using this table does not need to handle like check cache/db. For this, 
> we can update get API in the table to handle the cache.
>  
> This Jira will implement:
>  # Cache as a part of each Table.
>  # Uses this cache in get().
>  # Exposes api for cleanup, add entries to cache.
> Usage to add the entries in to cache 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