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

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

                Author: ASF GitHub Bot
            Created on: 07/Aug/19 17:26
            Start Date: 07/Aug/19 17:26
    Worklog Time Spent: 10m 
      Work Description: bharatviswa504 commented on pull request #1247: 
HDDS-1926. The new caching layer is used for old OM requests but not updated.
URL: https://github.com/apache/hadoop/pull/1247
 
 
   When I am trying to add a test with restart manager, and then try to create 
the same volume, I am getting some NetUtils EOF exception. Will try to see why 
it is causing, if I am able to solve this issue, will post an Integration test 
for the scenario mentioned in the Jira.
 
----------------------------------------------------------------
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:
[email protected]


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

            Worklog Id:     (was: 290617)
            Time Spent: 10m
    Remaining Estimate: 0h

> The new caching layer is used for old OM requests but not updated
> -----------------------------------------------------------------
>
>                 Key: HDDS-1926
>                 URL: https://issues.apache.org/jira/browse/HDDS-1926
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>          Components: om
>            Reporter: Elek, Marton
>            Assignee: Bharat Viswanadham
>            Priority: Blocker
>              Labels: pull-request-available
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> HDDS-1499 introduced a new caching layer together with a double-buffer based 
> db writer to support OM HA.
> TLDR: I think the caching layer is not updated for new volume creation. And 
> (slightly related to this problem) I suggest to separated the TypedTable and 
> the caching layer.
> ## How to reproduce the problem?
> 1. Start a docker compose cluster
> 2. Create one volume (let's say `/vol1`)
> 3. Restart the om (!)
> 4. Try to create an _other_ volume twice!
> ```
> bash-4.2$ ozone sh volume create /vol2
> 2019-08-07 12:29:47 INFO  RpcClient:288 - Creating Volume: vol2, with hadoop 
> as owner.
> bash-4.2$ ozone sh volume create /vol2
> 2019-08-07 12:29:50 INFO  RpcClient:288 - Creating Volume: vol2, with hadoop 
> as owner.
> ```
> Expected behavior is an error:
> {code}
> bash-4.2$ ozone sh volume create /vol1
> 2019-08-07 09:48:39 INFO  RpcClient:288 - Creating Volume: vol1, with hadoop 
> as owner.
> bash-4.2$ ozone sh volume create /vol1
> 2019-08-07 09:48:42 INFO  RpcClient:288 - Creating Volume: vol1, with hadoop 
> as owner.
> VOLUME_ALREADY_EXISTS 
> {code}
> The problem is that the new cache is used even for the old code path 
> (TypedTable):
> {code}
>  @Override
>   public VALUE get(KEY key) throws IOException {
>     // Here the metadata lock will guarantee that cache is not updated for 
> same
>     // key during get key.
>     CacheResult<CacheValue<VALUE>> cacheResult =
>         cache.lookup(new CacheKey<>(key));
>     if (cacheResult.getCacheStatus() == EXISTS) {
>       return cacheResult.getValue().getCacheValue();
>     } else if (cacheResult.getCacheStatus() == NOT_EXIST) {
>       return null;
>     } else {
>       return getFromTable(key);
>     }
>   }
> {code}
> For volume table after the FIRST start it always returns with 
> `getFromTable(key)` due to the condition in the `TableCacheImpl.lookup`:
> {code}
>   public CacheResult<CACHEVALUE> lookup(CACHEKEY cachekey) {
>     if (cache.size() == 0) {
>       return new CacheResult<>(CacheResult.CacheStatus.MAY_EXIST,
>           null);
>     }
> {code}
> But after a restart the cache is pre-loaded by the TypedTable.constructor. 
> After the restart, the real caching logic will be used (as cache.size()>0), 
> which cause a problem as the cache is NOT updated from the old code path.
> An additional problem is that the cache is turned on for all the metadata 
> table even if the cache is not required... 
> ## Proposed solution
> As I commented at HDDS-1499 this caching layer is not a "traditional cache". 
> It's not updated during the typedTable.put() call but updated by a separated 
> component during double-buffer flash.
> I would suggest to remove the cache related methods from TypedTable (move to 
> a separated implementation). I think this kind of caching can be independent 
> from the TypedTable implementation. We can continue to use the simple 
> TypedTable everywhere where we don't need to use any kind of caching.
> For caching we can use a separated object. It would make it more visible that 
> the cache should always be updated manually all the time. This separated 
> caching utility may include a reference to the original TypedTable/Table. 
> With this approach we can separate the different responsibilities but provide 
> the same functionality.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to