zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From lvfangmin <...@git.apache.org>
Subject [GitHub] zookeeper pull request #684: ZOOKEEPER-3180: Add response cache to improve t...
Date Tue, 04 Dec 2018 18:57:24 GMT
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/684#discussion_r238794022
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/ResponseCache.java
---
    @@ -0,0 +1,84 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import java.util.Collections;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.data.Stat;
    +
    +@SuppressWarnings("serial")
    +public class ResponseCache {
    +    private static final int DEFAULT_RESPONSE_CACHE_SIZE = 400;
    +
    +    private static class Entry {
    +        public Stat stat;
    +        public byte[] data;
    +    }
    +
    +    private Map<String, Entry> cache = Collections.synchronizedMap(
    +        new LRUCache<String, Entry>(getResponseCacheSize()));
    +
    +    public ResponseCache() {
    +    }
    +
    +    public void put(String path, byte[] data, Stat stat) {
    +        Entry entry = new Entry();
    +        entry.data = data;
    +        entry.stat = stat;
    +        cache.put(path, entry);
    +    }
    +
    +    public byte[] get(String key, Stat stat) {
    +        Entry entry = cache.get(key);
    +        if (entry == null) {
    +            return null;
    +        }
    +        if (!stat.equals(entry.stat)) {
    --- End diff --
    
    Currently, he stat is also part of the cached data, when it's changed we need to invalidate
this as well.
    
    We can separate the cache for data and stat, but we didn't see that's necessary for now,
this is mainly for improving the serializing performance and GC effort with large znode data
size.


---

Mime
View raw message