add retry strategy in cache sync

Signed-off-by: shaofengshi <shaofeng...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/7da2cf8a
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/7da2cf8a
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/7da2cf8a

Branch: refs/heads/ranger
Commit: 7da2cf8a1818ab78c3df679c95b44a332c4d4bd0
Parents: 7bdce5e
Author: lichao <cha...@mobvoi.com>
Authored: Thu Sep 7 19:45:45 2017 +0800
Committer: shaofengshi <shaofeng...@apache.org>
Committed: Fri Sep 8 22:36:48 2017 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |  6 ++++-
 .../main/resources/kylin-defaults.properties    |  3 +++
 .../kylin/metadata/cachesync/Broadcaster.java   | 27 +++++++++++++++++++-
 3 files changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7da2cf8a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java 
b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 749b515..98bbb8b 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -253,6 +253,10 @@ abstract public class KylinConfigBase implements 
Serializable {
         return StorageURL.valueOf(getOptional("kylin.metadata.url", 
"kylin_metadata@hbase"));
     }
 
+    public int getCacheSyncRetrys(){
+        return 
Integer.parseInt(getOptional("kylin.metadata.sync.retries","3"));
+    }
+
     // for test only
     public void setMetadataUrl(String metadataUrl) {
         setProperty("kylin.metadata.url", metadataUrl);
@@ -342,7 +346,7 @@ abstract public class KylinConfigBase implements 
Serializable {
     public String getSegmentAdvisor() {
         return getOptional("kylin.cube.segment-advisor", 
"org.apache.kylin.cube.CubeSegmentAdvisor");
     }
-    
+
     public double getJobCuboidSizeRatio() {
         return 
Double.parseDouble(getOptional("kylin.cube.size-estimate-ratio", "0.25"));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/7da2cf8a/core-common/src/main/resources/kylin-defaults.properties
----------------------------------------------------------------------
diff --git a/core-common/src/main/resources/kylin-defaults.properties 
b/core-common/src/main/resources/kylin-defaults.properties
index a3277ce..1443779 100644
--- a/core-common/src/main/resources/kylin-defaults.properties
+++ b/core-common/src/main/resources/kylin-defaults.properties
@@ -20,6 +20,9 @@
 # The metadata store in hbase
 kylin.metadata.url=kylin_metadata@hbase
 
+# metadata cache sync retry times
+kylin.metadata.sync.retries=3
+
 # Working folder in HDFS, better be qualified absolute path, make sure user 
has the right permission to this directory
 kylin.env.hdfs-working-dir=/kylin
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/7da2cf8a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
----------------------------------------------------------------------
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index 4b0ef57..8629fe2 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -111,6 +111,7 @@ public class Broadcaster {
 
     private Broadcaster(final KylinConfig config) {
         this.config = config;
+        final int retryLimitTimes = config.getCacheSyncRetrys();
 
         final String[] nodes = config.getRestServers();
         if (nodes == null || nodes.length < 1) {
@@ -128,6 +129,12 @@ public class Broadcaster {
                 while (true) {
                     try {
                         final BroadcastEvent broadcastEvent = 
broadcastEvents.takeFirst();
+                        
broadcastEvent.setRetryTime(broadcastEvent.getRetryTime() + 1);
+                        if (broadcastEvent.getRetryTime() > retryLimitTimes) {
+                            logger.info("broadcastEvent retry up to limit 
times, broadcastEvent:{}", broadcastEvent);
+                            continue;
+                        }
+
                         String[] restServers = config.getRestServers();
                         logger.debug("Servers in the cluster: " + 
Arrays.toString(restServers));
                         for (final String node : restServers) {
@@ -145,7 +152,16 @@ public class Broadcaster {
                                         
restClientMap.get(node).wipeCache(broadcastEvent.getEntity(),
                                                 broadcastEvent.getEvent(), 
broadcastEvent.getCacheKey());
                                     } catch (IOException e) {
-                                        logger.warn("Thread failed during wipe 
cache at " + broadcastEvent, e);
+                                        logger.warn("Thread failed during wipe 
cache at {}, error msg: {}",
+                                                broadcastEvent, e);
+                                        // when sync failed, put back to queue
+                                        try {
+                                            
broadcastEvents.putLast(broadcastEvent);
+                                        } catch (InterruptedException ex) {
+                                            logger.warn(
+                                                    "error reentry failed 
broadcastEvent to queue, broacastEvent:{}, error: {} ",
+                                                    broadcastEvent, ex);
+                                        }
                                     }
                                 }
                             });
@@ -297,6 +313,7 @@ public class Broadcaster {
     }
 
     public static class BroadcastEvent {
+        private int retryTime;
         private String entity;
         private String event;
         private String cacheKey;
@@ -308,6 +325,14 @@ public class Broadcaster {
             this.cacheKey = cacheKey;
         }
 
+        public int getRetryTime() {
+            return retryTime;
+        }
+
+        public void setRetryTime(int retryTime) {
+            this.retryTime = retryTime;
+        }
+
         public String getEntity() {
             return entity;
         }

Reply via email to