Mukvin commented on code in PR #2015:
URL: https://github.com/apache/kylin/pull/2015#discussion_r1022364553


##########
cache/src/main/java/org/apache/kylin/cache/cachemanager/RemoteLocalFailOverCacheManager.java:
##########
@@ -70,7 +70,7 @@ void enableRemoteCacheManager() {
     }
 
     @VisibleForTesting
-    MemcachedCacheManager getRemoteCacheManager() {
+    AbstractRemoteCacheManager getRemoteCacheManager() {
         return remoteCacheManager;
     }
 }

Review Comment:
   there should be an empty line in the end of file



##########
server/src/main/resources/applicationContext.xml:
##########
@@ -118,6 +118,24 @@
                     <property name="hosts" 
value="${kylin.cache.memcached.hosts}" />
                 </bean>
         -->
+
+        <!-- redis cache config
+        <bean id="ehcache" 
class="org.springframework.cache.ehcache.EhCacheManagerFactoryBean"
+              p:configLocation="classpath:ehcache.xml" p:shared="true"/>
+        <bean id="remoteCacheManager" 
class="org.apache.kylin.cache.cachemanager.RedisManager" />
+        <bean id="localCacheManager" 
class="org.springframework.cache.ehcache.EhCacheCacheManager"
+              p:cacheManager-ref="ehcache"/>
+        <bean id="cacheManager" 
class="org.apache.kylin.cache.cachemanager.RemoteLocalFailOverCacheManager" />
+
+        <bean id="redisConfig" 
class="org.apache.kylin.cache.redis.RedisConfig">
+            <property name="appName" value="kylin" />
+            <property name="timeout" value="500" />

Review Comment:
   timeout should be using variables too.



##########
cache/src/test/java/org/apache/kylin/cache/redis/JedisPoolClientTest.java:
##########
@@ -0,0 +1,76 @@
+/*

Review Comment:
   Is this code of the file not formatted?



##########
cache/src/main/java/org/apache/kylin/cache/redis/AbstractRedisClient.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.kylin.cache.redis;
+
+import com.codahale.metrics.Gauge;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.directory.api.util.Strings;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.CompressionUtils;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.shaded.com.google.common.base.Charsets;
+import org.apache.kylin.shaded.com.google.common.base.Joiner;
+import org.apache.kylin.shaded.com.google.common.primitives.Ints;
+import org.apache.kylin.shaded.com.google.common.primitives.Shorts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.zip.DataFormatException;
+import static com.codahale.metrics.MetricRegistry.name;
+import static org.apache.kylin.metrics.lib.impl.MetricsSystem.Metrics;
+
+
+public abstract class AbstractRedisClient implements RedisClient {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(AbstractRedisClient.class);
+
+    protected final RedisConfig config;
+    protected final String redisPrefix;
+    protected final int compressThreshold;
+    protected final int timeToLiveSeconds;
+
+    protected final AtomicLong hitCount = new AtomicLong(0);
+    protected final AtomicLong missCount = new AtomicLong(0);
+    protected final AtomicLong readBytes = new AtomicLong(0);
+    protected final AtomicLong errorCount = new AtomicLong(0);
+    protected final AtomicLong putCount = new AtomicLong(0);
+    protected final AtomicLong putBytes = new AtomicLong(0);
+    protected final AtomicLong delCount = new AtomicLong(0);
+
+    public AbstractRedisClient(RedisConfig redisConfig) {
+        this.config = redisConfig;
+        compressThreshold = redisConfig.getMaxObjectSize() / 2;
+        redisPrefix = redisConfig.getPrefix();
+        timeToLiveSeconds = redisConfig.getTtl();
+
+        Map<String, String> metricsConfig = 
KylinConfig.getInstanceFromEnv().getKylinMetricsConf();
+        if ("true".equalsIgnoreCase(metricsConfig.get("redis.enabled"))) {
+            final String prefix = name(config.getRedisClientType().name(), 
redisPrefix);
+            Metrics.register(name(prefix, "hits"), new Gauge<Long>() {
+                @Override
+                public Long getValue() {
+                    return hitCount.longValue();
+                }
+            });
+
+            Metrics.register(name(prefix, "misses"), new Gauge<Long>() {
+                @Override
+                public Long getValue() {
+                    return missCount.longValue();
+                }
+            });
+
+            Metrics.register(name(prefix, "eviction-count"), new Gauge<Long>() 
{
+                @Override
+                public Long getValue() {
+                    return delCount.longValue();
+                }
+            });
+
+            Metrics.register(name(prefix, "put-count"), new Gauge<Long>() {
+                @Override
+                public Long getValue() {
+                    return putCount.longValue();
+                }
+            });
+        }
+    }
+
+    public abstract byte[] internalGet(String key);
+    public abstract void internalPut(String hashedKey, byte[] encodedValue, 
int expiration);
+    public abstract void internalDel(String key);
+
+    @Override
+    public void put(Object key, Object value) {
+        String keyString = serializeKey(key);
+        if (Strings.isEmpty(keyString)) {
+            return;
+        }
+        String hashKey = computeKeyHash(keyString);
+        byte[] encodedValue = encodeValue(keyString.getBytes(Charsets.UTF_8), 
serializeValue(value));
+        if (encodedValue.length > config.getMaxObjectSize()) {
+            logger.info("AbstractRedisClient#put, value oversize, value 
size:{}, maxObjectSize:{}", encodedValue.length, config.getMaxObjectSize());
+            return;
+        }
+        internalPut(hashKey, encodedValue, timeToLiveSeconds);
+        putCount.incrementAndGet();
+        putBytes.addAndGet(encodedValue.length);
+    }
+
+    @Override
+    public byte[] get(Object key) {
+        String keyString = serializeKey(key);
+        if (Strings.isEmpty(keyString)) {
+            return null;
+        }
+        byte[] result = internalGet(computeKeyHash(keyString));
+        if (result == null) {
+            missCount.incrementAndGet();
+        } else {
+            hitCount.incrementAndGet();
+            readBytes.addAndGet(result.length);
+        }
+        return decodeValue(keyString.getBytes(Charsets.UTF_8), result);
+    }
+
+    @Override
+    public void del(Object key) {
+        if (key == null) {
+            return;
+        }
+        String keyString = computeKeyHash(serializeKey(key));
+        if (Strings.isEmpty(keyString)) {
+            return;
+        }
+        internalDel(keyString);
+        delCount.incrementAndGet();
+    }
+
+    @Override
+    public String getName(){
+        return redisPrefix;
+    }
+
+    protected String serializeKey(Object key) {
+        try {
+            return JsonUtil.writeValueAsString(key);
+        } catch (JsonProcessingException e) {
+            logger.warn("Can not convert key to String.", e);
+        }
+        return null;
+    }
+
+    protected byte[] serializeValue(Object value) {
+        return SerializationUtils.serialize((Serializable) value);
+    }
+
+    protected byte[] encodeValue(byte[] key, byte[] valueB) {
+        byte[] compressed = null;
+        if (config.isEnableCompression() && (valueB.length + Ints.BYTES + 
key.length > compressThreshold)) {
+            try {
+                compressed = 
CompressionUtils.compress(ByteBuffer.allocate(Ints.BYTES + key.length + 
valueB.length)
+                        .putInt(key.length).put(key).put(valueB).array());
+            } catch (IOException e) {
+                compressed = null;
+                logger.warn("Compressing value bytes error.", e);
+            }
+        }
+        if (compressed != null) {
+            return ByteBuffer.allocate(Shorts.BYTES + 
compressed.length).putShort((short) 1).put(compressed).array();
+        } else {
+            return ByteBuffer.allocate(Shorts.BYTES + Ints.BYTES + key.length 
+ valueB.length).putShort((short) 0)
+                    .putInt(key.length).put(key).put(valueB).array();
+        }
+    }
+
+    protected byte[] decodeValue(byte[] key, byte[] valueE) {
+        if (valueE == null)
+            return null;
+        ByteBuffer buf = ByteBuffer.wrap(valueE);
+        short enableCompression = buf.getShort();
+        byte[] uncompressed = null;
+        if (enableCompression == 1) {

Review Comment:
   1 is magic number



##########
cache/src/main/java/org/apache/kylin/cache/cachemanager/RedisManager.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.kylin.cache.cachemanager;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.kylin.cache.redis.RedisClient;
+import org.apache.kylin.cache.redis.RedisConfig;
+import org.apache.kylin.cache.redis.jedis.JedisPoolClient;
+import org.apache.kylin.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.kylin.shaded.com.google.common.collect.Lists;
+import 
org.apache.kylin.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.cache.Cache;
+import org.springframework.cache.support.SimpleValueWrapper;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import static org.apache.kylin.cache.redis.RedisClientTypeEnum.JEDIS_POOL;

Review Comment:
   optimize imports?



##########
cache/src/main/java/org/apache/kylin/cache/redis/AbstractRedisClient.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.kylin.cache.redis;
+
+import com.codahale.metrics.Gauge;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.directory.api.util.Strings;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.CompressionUtils;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.shaded.com.google.common.base.Charsets;
+import org.apache.kylin.shaded.com.google.common.base.Joiner;
+import org.apache.kylin.shaded.com.google.common.primitives.Ints;
+import org.apache.kylin.shaded.com.google.common.primitives.Shorts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.zip.DataFormatException;
+import static com.codahale.metrics.MetricRegistry.name;
+import static org.apache.kylin.metrics.lib.impl.MetricsSystem.Metrics;

Review Comment:
   optimize imports?



##########
cache/src/main/java/org/apache/kylin/cache/cachemanager/MultiLevelCacheManager.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.kylin.cache.cachemanager;
+
+import org.apache.kylin.shaded.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.cache.Cache;
+import org.springframework.cache.CacheManager;
+import org.springframework.cache.support.AbstractCacheManager;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+
+public class MultiLevelCacheManager extends AbstractCacheManager {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(MultiLevelCacheManager.class);
+
+    @Autowired
+    private AbstractRemoteCacheManager remoteCacheManager;
+
+    @Autowired
+    private CacheManager localCacheManager;
+
+    @Override
+    protected Collection<? extends Cache> loadCaches() {
+        Cache successCache = new 
MultiLevelCacheManager.MultiLevelCacheAdaptor(remoteCacheManager, 
localCacheManager, CacheConstants.QUERY_CACHE);
+        addCache(successCache);

Review Comment:
   addCache() is a deprecated method.



##########
cache/src/test/java/org/apache/kylin/cache/redis/JedisPoolClientTest.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.kylin.cache.redis;
+
+import org.apache.kylin.cache.cachemanager.CacheConstants;
+import org.apache.kylin.cache.cachemanager.RedisManager;
+import org.apache.kylin.cache.redis.jedis.JedisPoolClient;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.shaded.com.google.common.base.Charsets;
+import org.apache.kylin.shaded.com.google.common.collect.Maps;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import java.util.Map;
+import static org.mockito.Mockito.when;
+
+public class JedisPoolClientTest extends LocalFileMetadataTestCase {
+
+    private Map<String, String> keyValueMap;
+
+    private RedisManager.RedisCacheAdaptor redisCacheAdaptor;
+
+
+    @Before
+    public void setUp() {
+        this.createTestMetadata();
+
+        keyValueMap = Maps.newHashMap();
+        keyValueMap.put("sql1", "value1");
+        keyValueMap.put("sql11", "value11");
+
+        RedisConfig redisConfig = new RedisConfig();
+        JedisPoolClient jedisPoolClient = new JedisPoolClient(redisConfig);
+        JedisPoolClient clientSpy = Mockito.spy(jedisPoolClient);
+        redisCacheAdaptor = new RedisManager.RedisCacheAdaptor(clientSpy, 
CacheConstants.QUERY_CACHE);
+
+        for (String key : keyValueMap.keySet()) {
+            String keyS = jedisPoolClient.serializeKey(key);
+            String hashedKey = jedisPoolClient.computeKeyHash(keyS);
+            String value = keyValueMap.get(key);
+            byte[] valueE = 
jedisPoolClient.encodeValue(keyS.getBytes(Charsets.UTF_8), 
jedisPoolClient.serializeValue(value));
+            when(clientSpy.internalGet(hashedKey)).thenReturn(valueE);
+        }
+
+    }
+
+    @After
+    public void after() {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGet() {
+        for (String key : keyValueMap.keySet()) {
+            Assert.assertEquals("The value should not change", 
keyValueMap.get(key), redisCacheAdaptor.get(key).get());
+        }
+    }
+

Review Comment:
   There may add other test methods about JedisPoolClient?



##########
cache/src/test/java/org/apache/kylin/cache/redis/JedisPoolClientTest.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.kylin.cache.redis;
+
+import org.apache.kylin.cache.cachemanager.CacheConstants;
+import org.apache.kylin.cache.cachemanager.RedisManager;
+import org.apache.kylin.cache.redis.jedis.JedisPoolClient;
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.shaded.com.google.common.base.Charsets;
+import org.apache.kylin.shaded.com.google.common.collect.Maps;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import java.util.Map;
+import static org.mockito.Mockito.when;
+
+public class JedisPoolClientTest extends LocalFileMetadataTestCase {
+
+    private Map<String, String> keyValueMap;
+
+    private RedisManager.RedisCacheAdaptor redisCacheAdaptor;
+
+
+    @Before
+    public void setUp() {
+        this.createTestMetadata();
+
+        keyValueMap = Maps.newHashMap();
+        keyValueMap.put("sql1", "value1");
+        keyValueMap.put("sql11", "value11");
+
+        RedisConfig redisConfig = new RedisConfig();
+        JedisPoolClient jedisPoolClient = new JedisPoolClient(redisConfig);
+        JedisPoolClient clientSpy = Mockito.spy(jedisPoolClient);
+        redisCacheAdaptor = new RedisManager.RedisCacheAdaptor(clientSpy, 
CacheConstants.QUERY_CACHE);
+
+        for (String key : keyValueMap.keySet()) {
+            String keyS = jedisPoolClient.serializeKey(key);
+            String hashedKey = jedisPoolClient.computeKeyHash(keyS);
+            String value = keyValueMap.get(key);
+            byte[] valueE = 
jedisPoolClient.encodeValue(keyS.getBytes(Charsets.UTF_8), 
jedisPoolClient.serializeValue(value));
+            when(clientSpy.internalGet(hashedKey)).thenReturn(valueE);
+        }
+
+    }
+
+    @After
+    public void after() {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testGet() {
+        for (String key : keyValueMap.keySet()) {
+            Assert.assertEquals("The value should not change", 
keyValueMap.get(key), redisCacheAdaptor.get(key).get());
+        }
+    }
+
+}

Review Comment:
   a file should be with an empty line at the end.



##########
cache/src/main/java/org/apache/kylin/cache/redis/AbstractRedisClient.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.kylin.cache.redis;
+
+import com.codahale.metrics.Gauge;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.directory.api.util.Strings;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.CompressionUtils;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.shaded.com.google.common.base.Charsets;
+import org.apache.kylin.shaded.com.google.common.base.Joiner;
+import org.apache.kylin.shaded.com.google.common.primitives.Ints;
+import org.apache.kylin.shaded.com.google.common.primitives.Shorts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.zip.DataFormatException;
+import static com.codahale.metrics.MetricRegistry.name;
+import static org.apache.kylin.metrics.lib.impl.MetricsSystem.Metrics;
+
+
+public abstract class AbstractRedisClient implements RedisClient {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(AbstractRedisClient.class);
+
+    protected final RedisConfig config;
+    protected final String redisPrefix;
+    protected final int compressThreshold;
+    protected final int timeToLiveSeconds;
+
+    protected final AtomicLong hitCount = new AtomicLong(0);
+    protected final AtomicLong missCount = new AtomicLong(0);
+    protected final AtomicLong readBytes = new AtomicLong(0);
+    protected final AtomicLong errorCount = new AtomicLong(0);
+    protected final AtomicLong putCount = new AtomicLong(0);
+    protected final AtomicLong putBytes = new AtomicLong(0);
+    protected final AtomicLong delCount = new AtomicLong(0);
+
+    public AbstractRedisClient(RedisConfig redisConfig) {
+        this.config = redisConfig;
+        compressThreshold = redisConfig.getMaxObjectSize() / 2;
+        redisPrefix = redisConfig.getPrefix();
+        timeToLiveSeconds = redisConfig.getTtl();
+
+        Map<String, String> metricsConfig = 
KylinConfig.getInstanceFromEnv().getKylinMetricsConf();
+        if ("true".equalsIgnoreCase(metricsConfig.get("redis.enabled"))) {

Review Comment:
   "true" is magic value



##########
cache/src/main/java/org/apache/kylin/cache/cachemanager/RedisManager.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.kylin.cache.cachemanager;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.kylin.cache.redis.RedisClient;
+import org.apache.kylin.cache.redis.RedisConfig;
+import org.apache.kylin.cache.redis.jedis.JedisPoolClient;
+import org.apache.kylin.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.kylin.shaded.com.google.common.collect.Lists;
+import 
org.apache.kylin.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.cache.Cache;
+import org.springframework.cache.support.SimpleValueWrapper;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import static org.apache.kylin.cache.redis.RedisClientTypeEnum.JEDIS_POOL;
+
+
+public class RedisManager extends AbstractRemoteCacheManager {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(RedisManager.class);
+    private static final Long ONE_MINUTE = 60 * 1000L;
+
+    @Autowired
+    private RedisConfig redisConfig;
+
+    private ScheduledExecutorService timer = 
Executors.newScheduledThreadPool(1,
+            new 
ThreadFactoryBuilder().setNameFormat("Redis-HealthChecker").build());
+    private AtomicBoolean clusterHealth = new AtomicBoolean(true);
+
+    @Override
+    protected Collection<? extends Cache> loadCaches() {
+        Cache successCache = new 
RedisManager.RedisCacheAdaptor(createRedisClient(redisConfig), 
CacheConstants.QUERY_CACHE);
+
+        addCache(successCache);
+
+        Collection<String> names = getCacheNames();
+        Collection<Cache> caches = Lists.newArrayList();
+        for (String name : names) {
+            caches.add(getCache(name));
+        }
+        timer.scheduleWithFixedDelay(new 
RedisManager.RedisClusterHealthChecker(), ONE_MINUTE, ONE_MINUTE,
+                TimeUnit.MILLISECONDS);
+        return caches;
+    }
+
+    public static class RedisCacheAdaptor implements Cache {
+
+        private RedisClient redisClient;
+        private String name;
+
+        public RedisCacheAdaptor(RedisClient redisClient, String name) {
+            this.redisClient = redisClient;
+            this.name = name;
+        }
+
+        @Override
+        public String getName() {
+            return name;
+        }
+
+        @Override
+        public Object getNativeCache() {
+            return redisClient;
+        }
+
+        @Override
+        public ValueWrapper get(Object key) {
+            byte[] value = redisClient.get(key);
+            if (value == null) {
+                return null;
+            }
+            return new 
SimpleValueWrapper(SerializationUtils.deserialize(value));
+        }
+
+        @Override
+        public <T> T get(Object key, Class<T> type) {
+            byte[] value = redisClient.get(key);
+            if (value == null) {
+                return null;
+            }
+            Object obj = SerializationUtils.deserialize(value);
+            if (obj != null && type != null && !type.isInstance(value)) {
+                throw new IllegalStateException(
+                        "Cached value is not of required type [" + 
type.getName() + "]: " + value);
+            }
+            return (T) obj;
+        }
+
+        @Override
+        public <T> T get(Object key, Callable<T> valueLoader) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void put(Object key, Object value) {
+            redisClient.put(key, value);
+        }
+
+        @Override
+        public ValueWrapper putIfAbsent(Object key, Object value) {
+            byte[] existing = redisClient.get(key);
+            if (existing == null) {
+                redisClient.put(key, value);
+                return null;
+            } else {
+                return new 
SimpleValueWrapper(SerializationUtils.deserialize(existing));
+            }
+        }
+
+        @Override
+        public void evict(Object key) {
+            redisClient.del(key);
+        }
+
+        @Override
+        public void clear() {
+            throw new UnsupportedOperationException();
+        }
+
+    }
+
+    protected RedisClient createRedisClient(RedisConfig redisConfig) {
+        RedisClient redisClient = null;
+        if (redisConfig.getRedisClientType() == JEDIS_POOL) {
+            redisClient = new JedisPoolClient(redisConfig);
+        }
+        assert redisClient != null;
+        return redisClient;
+    }
+
+    @Override
+    public boolean isClusterDown() {
+        return !clusterHealth.get();
+    }
+
+    @VisibleForTesting
+    void setClusterHealth(boolean ifHealth) {
+        clusterHealth.set(ifHealth);
+    }
+
+    private class RedisClusterHealthChecker implements Runnable {
+        @Override
+        public void run() {
+            Cache cache = getCache(CacheConstants.QUERY_CACHE);
+            RedisClient redisClient = (RedisClient) cache.getNativeCache();
+            clusterHealth.set(isConnected(redisClient));
+        }
+
+        private boolean isConnected(RedisClient redisClient) {
+            int success = 0;
+            int total = 5;

Review Comment:
   This is magic number



##########
cache/src/main/java/org/apache/kylin/cache/redis/AbstractRedisClient.java:
##########
@@ -0,0 +1,226 @@
+/*
+ * 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.kylin.cache.redis;
+
+import com.codahale.metrics.Gauge;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.directory.api.util.Strings;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.CompressionUtils;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.shaded.com.google.common.base.Charsets;
+import org.apache.kylin.shaded.com.google.common.base.Joiner;
+import org.apache.kylin.shaded.com.google.common.primitives.Ints;
+import org.apache.kylin.shaded.com.google.common.primitives.Shorts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.zip.DataFormatException;
+import static com.codahale.metrics.MetricRegistry.name;
+import static org.apache.kylin.metrics.lib.impl.MetricsSystem.Metrics;
+
+
+public abstract class AbstractRedisClient implements RedisClient {
+
+    private static final Logger logger = 
LoggerFactory.getLogger(AbstractRedisClient.class);
+
+    protected final RedisConfig config;
+    protected final String redisPrefix;
+    protected final int compressThreshold;
+    protected final int timeToLiveSeconds;
+
+    protected final AtomicLong hitCount = new AtomicLong(0);
+    protected final AtomicLong missCount = new AtomicLong(0);
+    protected final AtomicLong readBytes = new AtomicLong(0);
+    protected final AtomicLong errorCount = new AtomicLong(0);
+    protected final AtomicLong putCount = new AtomicLong(0);
+    protected final AtomicLong putBytes = new AtomicLong(0);
+    protected final AtomicLong delCount = new AtomicLong(0);
+
+    public AbstractRedisClient(RedisConfig redisConfig) {
+        this.config = redisConfig;
+        compressThreshold = redisConfig.getMaxObjectSize() / 2;
+        redisPrefix = redisConfig.getPrefix();
+        timeToLiveSeconds = redisConfig.getTtl();
+
+        Map<String, String> metricsConfig = 
KylinConfig.getInstanceFromEnv().getKylinMetricsConf();
+        if ("true".equalsIgnoreCase(metricsConfig.get("redis.enabled"))) {
+            final String prefix = name(config.getRedisClientType().name(), 
redisPrefix);
+            Metrics.register(name(prefix, "hits"), new Gauge<Long>() {
+                @Override
+                public Long getValue() {
+                    return hitCount.longValue();
+                }
+            });
+
+            Metrics.register(name(prefix, "misses"), new Gauge<Long>() {
+                @Override
+                public Long getValue() {
+                    return missCount.longValue();
+                }
+            });
+
+            Metrics.register(name(prefix, "eviction-count"), new Gauge<Long>() 
{
+                @Override
+                public Long getValue() {
+                    return delCount.longValue();
+                }
+            });
+
+            Metrics.register(name(prefix, "put-count"), new Gauge<Long>() {
+                @Override
+                public Long getValue() {
+                    return putCount.longValue();
+                }
+            });
+        }
+    }
+
+    public abstract byte[] internalGet(String key);
+    public abstract void internalPut(String hashedKey, byte[] encodedValue, 
int expiration);
+    public abstract void internalDel(String key);
+
+    @Override
+    public void put(Object key, Object value) {
+        String keyString = serializeKey(key);
+        if (Strings.isEmpty(keyString)) {
+            return;
+        }
+        String hashKey = computeKeyHash(keyString);
+        byte[] encodedValue = encodeValue(keyString.getBytes(Charsets.UTF_8), 
serializeValue(value));
+        if (encodedValue.length > config.getMaxObjectSize()) {
+            logger.info("AbstractRedisClient#put, value oversize, value 
size:{}, maxObjectSize:{}", encodedValue.length, config.getMaxObjectSize());
+            return;
+        }
+        internalPut(hashKey, encodedValue, timeToLiveSeconds);
+        putCount.incrementAndGet();
+        putBytes.addAndGet(encodedValue.length);
+    }
+
+    @Override
+    public byte[] get(Object key) {
+        String keyString = serializeKey(key);
+        if (Strings.isEmpty(keyString)) {
+            return null;
+        }
+        byte[] result = internalGet(computeKeyHash(keyString));
+        if (result == null) {
+            missCount.incrementAndGet();
+        } else {
+            hitCount.incrementAndGet();
+            readBytes.addAndGet(result.length);
+        }
+        return decodeValue(keyString.getBytes(Charsets.UTF_8), result);
+    }
+
+    @Override
+    public void del(Object key) {
+        if (key == null) {
+            return;
+        }
+        String keyString = computeKeyHash(serializeKey(key));
+        if (Strings.isEmpty(keyString)) {
+            return;
+        }
+        internalDel(keyString);
+        delCount.incrementAndGet();
+    }
+
+    @Override
+    public String getName(){
+        return redisPrefix;
+    }
+
+    protected String serializeKey(Object key) {
+        try {
+            return JsonUtil.writeValueAsString(key);
+        } catch (JsonProcessingException e) {
+            logger.warn("Can not convert key to String.", e);
+        }
+        return null;
+    }
+
+    protected byte[] serializeValue(Object value) {
+        return SerializationUtils.serialize((Serializable) value);
+    }
+
+    protected byte[] encodeValue(byte[] key, byte[] valueB) {
+        byte[] compressed = null;
+        if (config.isEnableCompression() && (valueB.length + Ints.BYTES + 
key.length > compressThreshold)) {
+            try {
+                compressed = 
CompressionUtils.compress(ByteBuffer.allocate(Ints.BYTES + key.length + 
valueB.length)
+                        .putInt(key.length).put(key).put(valueB).array());
+            } catch (IOException e) {
+                compressed = null;
+                logger.warn("Compressing value bytes error.", e);
+            }
+        }
+        if (compressed != null) {
+            return ByteBuffer.allocate(Shorts.BYTES + 
compressed.length).putShort((short) 1).put(compressed).array();
+        } else {
+            return ByteBuffer.allocate(Shorts.BYTES + Ints.BYTES + key.length 
+ valueB.length).putShort((short) 0)
+                    .putInt(key.length).put(key).put(valueB).array();
+        }
+    }
+
+    protected byte[] decodeValue(byte[] key, byte[] valueE) {
+        if (valueE == null)
+            return null;

Review Comment:
   format?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to