You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kylin.apache.org by GitBox <gi...@apache.org> on 2018/11/26 10:48:53 UTC

[GitHub] shaofengshi closed pull request #349: KYLIN-3694 Kylin On Druid Storage

shaofengshi closed pull request #349: KYLIN-3694 Kylin On Druid Storage
URL: https://github.com/apache/kylin/pull/349
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/assembly/pom.xml b/assembly/pom.xml
index dd3211a7cb..51e7658e05 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -50,6 +50,10 @@
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-storage-hbase</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage-druid</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-engine-mr</artifactId>
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 6092834e68..c72db0517f 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
@@ -24,6 +24,8 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Locale;
+import java.net.MalformedURLException;
+import java.net.URL;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
@@ -175,6 +177,14 @@ protected Properties getRawAllProperties() {
         return result;
     }
 
+    final protected String[] getRequiredStringArray(String prop) {
+        String[] result = getOptionalStringArray(prop, new String[0]);
+        if (result.length == 0) {
+            throw new IllegalArgumentException("missing '" + prop + "' in conf/kylin.properties");
+        }
+        return result;
+    }
+
     final protected String[] getOptionalStringArray(String prop, String[] dft) {
         final String property = getOptional(prop);
         if (!StringUtils.isBlank(property)) {
@@ -942,6 +952,7 @@ public String getJdbcSourceFieldDelimiter() {
         r.put(0, "org.apache.kylin.storage.hbase.HBaseStorage");
         r.put(1, "org.apache.kylin.storage.hybrid.HybridStorage");
         r.put(2, "org.apache.kylin.storage.hbase.HBaseStorage");
+        r.put(5, "org.apache.kylin.storage.druid.DruidStorage");
         r.putAll(convertKeyToInteger(getPropertiesByPrefix("kylin.storage.provider.")));
         return r;
     }
@@ -1120,6 +1131,81 @@ public int getHBaseReplicationScope() {
         return Integer.parseInt(getOptional("kylin.storage.hbase.replication-scope", "0"));
     }
 
+    // ============================================================================
+    // STORAGE.DRUID
+    // ============================================================================
+
+    public String getDruidHdfsLocation() {
+        return getOptional("kylin.storage.druid.hdfs-location");
+    }
+
+    public double getDruidShardCutGB() {
+        return Double.valueOf(getOptional("kylin.storage.druid.shard-cut-gb", "0.5"));
+    }
+
+    public short getDruidMinShardCount() {
+        short val = Short.valueOf(getOptional("kylin.storage.druid.min-shard-count", "1"));
+        return (short) Math.max(val, 1);
+    }
+
+    public short getDruidMaxShardCount() {
+        short val = Short.valueOf(getOptional("kylin.storage.druid.max-shard-count", "100"));
+        return (short) Math.min(val, Short.MAX_VALUE);
+    }
+
+    public String[] getDruidCoordinatorAddresses() {
+        String[] hosts = getRequiredStringArray("kylin.storage.druid.coordinator-addresses");
+        for (String host : hosts) {
+            try {
+                new URL(host);
+            } catch (MalformedURLException e) {
+                throw new IllegalArgumentException("Invalid valuefor kylin.storage.druid.coordinator-addresses");
+            }
+        }
+        return hosts;
+    }
+
+    public String getDruidBrokerHost() {
+        return getRequired("kylin.storage.druid.broker-host");
+    }
+
+    public String getDruidMysqlUrl() {
+        return getRequired("kylin.storage.druid.mysql-url");
+    }
+
+    public String getDruidMysqlUser() {
+        return getRequired("kylin.storage.druid.mysql-user");
+    }
+
+    public String getDruidMysqlPassword() {
+        return getRequired("kylin.storage.druid.mysql-password");
+    }
+
+    public String getDruidMysqlSegTabel() {
+        return getRequired("kylin.storage.druid.mysql-seg-table");
+    }
+
+    public int getDruidReducerThreadsNum() {
+        return Integer.parseInt(getOptional("kylin.storage.druid.reducer-thread-num", "1"));
+    }
+
+    public Map<String, String> getDruidMRConfigOverride() {
+        return getPropertiesByPrefix("kylin.engine.mr.druid-config-override.");
+    }
+
+    public String getDruidTierName() {
+        return getOptional("kylin.storage.druid.tier-name", "_default_tier");
+    }
+
+    public int getDruidReplicationNum() {
+        return Integer.parseInt(getOptional("kylin.storage.druid.replication-num", "2"));
+    }
+
+    public int getDruidDerivedInThreshold() {
+        return Integer.parseInt(getOptional("kylin.storage.druid.derived-filter-translation-threshold", "1000"));
+    }
+
+
     // ============================================================================
     // ENGINE.MR
     // ============================================================================
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index 9f95a7c316..33dd5e0c73 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -29,7 +29,6 @@
 import java.util.regex.Pattern;
 
 import javax.xml.bind.DatatypeConverter;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
@@ -278,7 +277,7 @@ public HttpResponse query(String sql, String project) throws IOException {
         HashMap<String, String> paraMap = new HashMap<String, String>();
         paraMap.put("sql", sql);
         paraMap.put("project", project);
-        String jsonMsg = new ObjectMapper().writeValueAsString(paraMap);
+        String jsonMsg = JsonUtil.writeValueAsString(paraMap);
         post.setEntity(new StringEntity(jsonMsg, "UTF-8"));
         HttpResponse response = client.execute(post);
         return response;
@@ -328,12 +327,29 @@ public String getLookupSnapshotCacheState(String lookupTableName, String snapsho
         return content;
     }
 
+    public void updateCube(String project, String cubeName, String cubeDescData) throws IOException {
+        HttpPut put = newPut(baseUrl + "/cubes");
+        HashMap<String, String> paraMap = new HashMap<>();
+        paraMap.put("project", project);
+        paraMap.put("cubeName", cubeName);
+        paraMap.put("cubeDescData", cubeDescData);
+        put.setEntity(new StringEntity(JsonUtil.writeValueAsString(paraMap), "UTF-8"));
+
+        HttpResponse response = client.execute(put);
+        HashMap respBody = dealResponse(response);
+        boolean success = (boolean) respBody.get("successful");
+        String errorMessage = (String) respBody.get("message");
+        if (!success) {
+            throw new IOException("Failed to edit cube: " + errorMessage);
+        }
+    }
+
     private HashMap dealResponse(HttpResponse response) throws IOException {
         if (response.getStatusLine().getStatusCode() != 200) {
             throw new IOException("Invalid response " + response.getStatusLine().getStatusCode());
         }
         String result = getContent(response);
-        HashMap resultMap = new ObjectMapper().readValue(result, HashMap.class);
+        HashMap resultMap = JsonUtil.readValue(result, HashMap.class);
         return resultMap;
     }
 
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
index a597780df5..36e803cecb 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
@@ -45,14 +45,14 @@ public static void writeShort(short num, byte[] bytes, int offset, int size) {
             num >>>= 8;
         }
     }
-
+    
     public static byte[] writeShort(short num) {
         byte[] output = new byte[Shorts.BYTES];
         writeShort(num, output, 0, output.length);
         return output;
     }
 
-    public static long readShort(byte[] bytes, int offset, int size) {
+    public static short readShort(byte[] bytes, int offset, int size) {
         short num = 0;
         for (int i = offset, n = offset + size; i < n; i++) {
             num <<= 8;
@@ -61,6 +61,7 @@ public static long readShort(byte[] bytes, int offset, int size) {
         return num;
     }
 
+
     public static short readShort(byte[] bytes) {
         return (short) readShort(bytes, 0, Shorts.BYTES);
     }
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java
index bf3f351440..3b4f5bce68 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/TimeUtil.java
@@ -35,6 +35,9 @@ private TimeUtil() {
     private static long ONE_HOUR_TS = 60 * ONE_MINUTE_TS;
     private static long ONE_DAY_TS = 24 * ONE_HOUR_TS;
 
+    /** The julian date of the epoch, 1970-01-01. */
+    public static final int EPOCH_JULIAN = 2440588;
+
     public static long getMinuteStart(long ts) {
         return ts / ONE_MINUTE_TS * ONE_MINUTE_TS;
     }
@@ -83,4 +86,54 @@ public static long getYearStart(long ts) {
         return calendar.getTimeInMillis();
     }
 
+    /**
+     * SQL extract(timeUnit from dateTime) function.
+     * Adapted from avatica DateTimeUtils.unixDateExtract.
+     *
+     * @param timeUnit one of YEAR, MONTH, DAY
+     * @param dateTime parsable date time string
+     * @return value of the `timeUnit` field from `dateTime`
+     */
+    public static long extract(String timeUnit, String dateTime) {
+        long millis = DateFormat.stringToMillis(dateTime);
+        int unixDate = (int) (millis / ONE_DAY_TS);
+        return julianExtract(timeUnit, unixDate + EPOCH_JULIAN);
+    }
+
+    private static int julianExtract(String timeUnit, int julian) {
+        // this shifts the epoch back to astronomical year -4800 instead of the
+        // start of the Christian era in year AD 1 of the proleptic Gregorian
+        // calendar.
+        int j = julian + 32044;
+        int g = j / 146097;
+        int dg = j % 146097;
+        int c = (dg / 36524 + 1) * 3 / 4;
+        int dc = dg - c * 36524;
+        int b = dc / 1461;
+        int db = dc % 1461;
+        int a = (db / 365 + 1) * 3 / 4;
+        int da = db - a * 365;
+
+        // integer number of full years elapsed since March 1, 4801 BC
+        int y = g * 400 + c * 100 + b * 4 + a;
+        // integer number of full months elapsed since the last March 1
+        int m = (da * 5 + 308) / 153 - 2;
+        // number of days elapsed since day 1 of the month
+        int d = da - (m + 4) * 153 / 5 + 122;
+        int year = y - 4800 + (m + 2) / 12;
+        int month = (m + 2) % 12 + 1;
+        int day = d + 1;
+
+        if ("YEAR".equals(timeUnit)) {
+            return year;
+        }
+        if ("MONTH".equals(timeUnit)) {
+            return month;
+        }
+        if ("DAY".equals(timeUnit)) {
+            return day;
+        }
+        throw new IllegalArgumentException("Invalid TimeUnit: " + timeUnit);
+    }
+
 }
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 07f429e84a..c593686707 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -236,6 +236,10 @@ public long getCreateTimeUTC() {
         return createTimeUTC;
     }
 
+    public String getCreateTimeUTCStr() {
+        return "" + createTimeUTC;
+    }
+
     public void setCreateTimeUTC(long createTimeUTC) {
         this.createTimeUTC = createTimeUTC;
     }
@@ -523,6 +527,10 @@ public void setCuboidShardNums(Map<Long, Short> newCuboidShards) {
         this.cuboidShardNums = newCuboidShards;
     }
 
+    public int getTotalShards() {
+        return totalShards;
+    }
+
     public int getTotalShards(long cuboidId) {
         if (totalShards > 0) {
             return totalShards;
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index 699b8653cf..09dc0c2c3e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -73,7 +73,7 @@ public static Cuboid findById(CubeSegment cubeSegment, long cuboidID) {
     }
 
     @VisibleForTesting
-    static Cuboid findById(CubeDesc cubeDesc, long cuboidID) {
+    public static Cuboid findById(CubeDesc cubeDesc, long cuboidID) {
         return findById(cubeDesc.getInitialCuboidScheduler(), cuboidID);
     }
 
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 4838e3c7e0..c8f9dd6e90 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -186,6 +186,8 @@ public String toString() {
     private int storageType = IStorageAware.ID_HBASE;
     @JsonProperty("override_kylin_properties")
     private LinkedHashMap<String, String> overrideKylinProps = new LinkedHashMap<String, String>();
+    @JsonProperty("online_version")
+    private int onlineVersion = 0;
 
     @JsonProperty("partition_offset_start")
     @JsonInclude(JsonInclude.Include.NON_EMPTY)
@@ -337,6 +339,18 @@ public DeriveInfo getHostInfo(TblColRef derived) {
         return result;
     }
 
+    public List<DeriveInfo> getDeriveInfos(List<TblColRef> rowCols) {
+        List<DeriveInfo> result = new ArrayList<>();
+        for (Entry<Array<TblColRef>, List<DeriveInfo>> entry : hostToDerivedMap.entrySet()) {
+            Array<TblColRef> hostCols = entry.getKey();
+            boolean hostOnRow = rowCols.containsAll(Arrays.asList(hostCols.data));
+            if (hostOnRow) {
+                result.addAll(entry.getValue());
+            }
+        }
+        return result;
+    }
+
     public String getResourcePath() {
         return concatResourcePath(resourceName());
     }
@@ -467,9 +481,20 @@ public void setStatusNeedNotify(List<String> statusNeedNotify) {
         return overrideKylinProps;
     }
 
-    private void setOverrideKylinProps(LinkedHashMap<String, String> overrideKylinProps) {
+    public void setOverrideKylinProps(LinkedHashMap<String, String> overrideKylinProps) {
         this.overrideKylinProps = overrideKylinProps;
     }
+    public synchronized void addOnlineVersion() {
+        onlineVersion ++;
+    }
+
+    public void setOnlineVersion(int onlineVersion) {
+        this.onlineVersion = onlineVersion;
+    }
+
+    public int getOnlineVersion() {
+        return onlineVersion;
+    }
 
     public List<Set<String>> getMandatoryDimensionSetList() {
         return mandatoryDimensionSetList;
@@ -570,8 +595,7 @@ public String calculateSignature() {
                     .append(JsonUtil.writeValueAsString(this.measures)).append("|")//
                     .append(JsonUtil.writeValueAsString(this.rowkey)).append("|")//
                     .append(JsonUtil.writeValueAsString(this.aggregationGroups)).append("|")//
-                    .append(JsonUtil.writeValueAsString(this.hbaseMapping)).append("|")//
-                    .append(JsonUtil.writeValueAsString(this.storageType)).append("|");
+                    .append(JsonUtil.writeValueAsString(this.hbaseMapping)).append("|");
 
             if (mandatoryDimensionSetList != null && !mandatoryDimensionSetList.isEmpty()) {
                 for (Set<String> mandatoryDimensionSet : mandatoryDimensionSetList) {
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/algorithm/CuboidStatsUtilTest.java b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/algorithm/CuboidStatsUtilTest.java
index ba8833afe8..c3642cb22d 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/cuboid/algorithm/CuboidStatsUtilTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/cuboid/algorithm/CuboidStatsUtilTest.java
@@ -22,6 +22,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -147,13 +148,12 @@ public void createDirectChildrenCacheTest() {
     @Test
     public void createDirectChildrenCacheStressTest() {
         Stopwatch sw = new Stopwatch();
-        sw.start();
         Set<Long> cuboidSet = generateMassCuboidSet();
-        System.out.println("Time elapsed for creating sorted cuboid list: " + sw.elapsedMillis());
+        System.out.println("Time elapsed for creating sorted cuboid list: " + sw.elapsed(TimeUnit.MILLISECONDS));
         sw.reset();
         sw.start();
         checkDirectChildrenCacheStressTest(CuboidStatsUtil.createDirectChildrenCache(cuboidSet));
-        System.out.println("Time elapsed for creating direct children cache: " + sw.elapsedMillis());
+        System.out.println("Time elapsed for creating direct children cache: " + sw.elapsed(TimeUnit.MILLISECONDS));
         sw.stop();
     }
 
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
index 57ea26c9a4..0e15687d7f 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -27,6 +27,7 @@
 import java.util.Random;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.measure.MeasureAggregator;
@@ -166,7 +167,8 @@ public void testEstimateBitmapMemSize() {
         }
     }
 
-    @Test
+    // Ignore because of OOM
+    // @Test
     public void testEstimateMemSize() throws InterruptedException {
         int scale = Integer.parseInt(System.getProperty("scale", "1"));
         scale = Math.max(1, Math.min(10, scale));
@@ -203,12 +205,12 @@ public int compare(byte[] o1, byte[] o2) {
             if ((i + 1) % reportInterval == 0) {
                 stopwatch.start();
                 long estimateBytes = GTAggregateScanner.estimateSizeOfAggrCache(key, values, map.size());
-                estimateMillis += stopwatch.elapsedMillis();
+                estimateMillis += stopwatch.elapsed(TimeUnit.MILLISECONDS);
                 stopwatch.reset();
 
                 stopwatch.start();
                 long actualBytes = meter.measureDeep(map);
-                actualMillis += stopwatch.elapsedMillis();
+                actualMillis += stopwatch.elapsed(TimeUnit.MILLISECONDS);
                 stopwatch.reset();
 
                 System.out.printf(Locale.ROOT, "%,15d %,15d %,15d %,15d %,15d\n", map.size(), estimateBytes,
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
index 2d73e59a4b..64bb32075f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
@@ -81,14 +81,18 @@ public int getMeasuresCount() {
     }
 
     public int[] getPeekLength(ByteBuffer buf) {
-        int[] length = new int[nMeasures];
+        int[] result = new int[nMeasures];
+        getPeekLength(buf, result);
+        return result;
+    }
+
+    public void getPeekLength(ByteBuffer buf, int[] result) {
         int offset = 0;
         for (int i = 0; i < nMeasures; i++) {
-            length[i] = serializers[i].peekLength(buf);
-            offset += length[i];
+            result[i] = serializers[i].peekLength(buf);
+            offset += result[i];
             buf.position(offset);
         }
-        return length;
     }
 
     public void decode(ByteBuffer buf, Object[] result) {
@@ -97,5 +101,9 @@ public void decode(ByteBuffer buf, Object[] result) {
             result[i] = serializers[i].deserialize(buf);
         }
     }
+    
+    public Object decode(ByteBuffer buf, int index) {
+        return serializers[index].deserialize(buf);
+    }
 
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java
index 9929e24342..9e4c038a7e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java
@@ -60,6 +60,10 @@ private MutableRoaringBitmap getMutableBitmap() {
         return result;
     }
 
+    public ImmutableRoaringBitmap getImmutableBimap() {
+        return bitmap;
+    }
+
     @Override
     public void add(int value) {
         getMutableBitmap().add(value);
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java
index 8ab908ab15..22ba45f284 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.measure.bitmap;
 
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
 import org.roaringbitmap.buffer.MutableRoaringBitmap;
 
 import java.io.IOException;
@@ -25,7 +26,7 @@
 import java.nio.ByteBuffer;
 
 public class RoaringBitmapCounterFactory implements BitmapCounterFactory, Serializable {
-    public static final BitmapCounterFactory INSTANCE = new RoaringBitmapCounterFactory();
+    public static final RoaringBitmapCounterFactory INSTANCE = new RoaringBitmapCounterFactory();
 
     private RoaringBitmapCounterFactory() {}
 
@@ -50,4 +51,8 @@ public BitmapCounter newBitmap(ByteBuffer in) throws IOException {
         counter.readFields(in);
         return counter;
     }
+
+    public BitmapCounter newBitmap(ImmutableRoaringBitmap bitmap) {
+        return new RoaringBitmapCounter(bitmap);
+    }
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CaseTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CaseTupleFilter.java
index 90832127b8..c81ee5a43f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CaseTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CaseTupleFilter.java
@@ -150,4 +150,8 @@ public TupleFilter acceptOptimizeTransformer(FilterOptimizeTransformer transform
         return transformer.visit(this);
     }
 
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitCase(this);
+    }
+
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ColumnTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ColumnTupleFilter.java
index 6d3d5414fb..92529e0d8a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ColumnTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ColumnTupleFilter.java
@@ -155,4 +155,9 @@ public void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
             this.columnRef = column.getRef();
         }
     }
+
+    @Override
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitColumn(this);
+    }
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
index 1c1c409024..9193761b56 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
@@ -255,6 +255,10 @@ public CompareResultType getCompareResultType() {
         return CompareResultType.Unknown;
     }
 
+    public boolean columnMatchSingleValue() {
+        return column != null && operator == FilterOperatorEnum.EQ && conditionValues.size() == 1;
+    }
+
     @SuppressWarnings({ "unchecked", "rawtypes" })
     @Override
     public void serialize(IFilterCodeSystem cs, ByteBuffer buffer) {
@@ -320,4 +324,9 @@ public int hashCode() {
         result = 31 * result + (dynamicVariables != null ? dynamicVariables.hashCode() : 0);
         return result;
     }
+
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitCompare(this);
+    }
+
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ConstantTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ConstantTupleFilter.java
index e4f8b2e755..5263dc2f4f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ConstantTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ConstantTupleFilter.java
@@ -122,4 +122,10 @@ public void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
     @Override public int hashCode() {
         return constantValues.hashCode();
     }
+
+    @Override
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitConstant(this);
+    }
+
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/DynamicTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/DynamicTupleFilter.java
index d9dc52a1b3..d95d3d064e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/DynamicTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/DynamicTupleFilter.java
@@ -78,4 +78,9 @@ public void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
         this.variableName = BytesUtil.readUTFString(buffer);
     }
 
+    @Override
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitDynamic(this);
+    }
+
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ExtractTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ExtractTupleFilter.java
index 8c2ba942f6..dd72838e8d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ExtractTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/ExtractTupleFilter.java
@@ -122,4 +122,9 @@ public void serialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
     public void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
     }
 
+    @Override
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitExtract(this);
+    }
+
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
index 72c247a108..4d180c6659 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/FunctionTupleFilter.java
@@ -25,4 +25,9 @@
     protected FunctionTupleFilter(List<TupleFilter> filters, FilterOperatorEnum op) {
         super(filters, op);
     }
+
+    @Override
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitFunction(this);
+    }
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
index f0c825f2a2..187dbf1966 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/LogicalTupleFilter.java
@@ -196,4 +196,9 @@ public boolean equals(Object other) {
     public int hashCode() {
         return (operator == null ? 0 : operator.hashCode()) + 31 * this.children.hashCode();
     }
+
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitLogical(this);
+    }
+
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
index 672aba0bda..ff42e72244 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
@@ -350,6 +350,8 @@ private void findMustTrueCompareFilters(TupleFilter filter, Set<CompareTupleFilt
 
     public abstract void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer);
 
+    public abstract <R> R accept(TupleFilterVisitor<R> visitor);
+
     public static boolean isEvaluableRecursively(TupleFilter filter) {
         if (filter == null)
             return true;
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
index d434f0b3e4..3ec972acaa 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
@@ -179,9 +179,6 @@ private static TupleFilter createTupleFilter(int opVal) {
         case ISNOTNULL:
             filter = new CompareTupleFilter(op);
             break;
-        case EXTRACT:
-            filter = new ExtractTupleFilter(op);
-            break;
         case CASE:
             filter = new CaseTupleFilter();
             break;
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor.java
new file mode 100644
index 0000000000..f31844cea2
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor.java
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.filter;
+
+public interface TupleFilterVisitor<R> {
+
+    R visitCase(CaseTupleFilter filter);
+
+    R visitColumn(ColumnTupleFilter filter);
+
+    R visitCompare(CompareTupleFilter filter);
+
+    R visitConstant(ConstantTupleFilter filter);
+
+    R visitDynamic(DynamicTupleFilter filter);
+
+    R visitFunction(FunctionTupleFilter filter);
+
+    R visitLogical(LogicalTupleFilter filter);
+
+    R visitExtract(ExtractTupleFilter filter);
+
+    R visitUnsupported(UnsupportedTupleFilter filter);
+}
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor2.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor2.java
new file mode 100644
index 0000000000..206a512e3c
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor2.java
@@ -0,0 +1,52 @@
+/*
+ * 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.metadata.filter;
+
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A simplified version of {@link TupleFilterVisitor}.
+ * @param <R>
+ */
+public interface TupleFilterVisitor2<R> {
+
+    R visitColumnCompare(CompareTupleFilter originFilter, TblColRef column, FilterOperatorEnum op, Set<?> values, Object firstValue);
+
+    R visitColumnLike(BuiltInFunctionTupleFilter originFilter, TblColRef column, String pattern, boolean reversed);
+
+    R visitColumnFunction(CompareTupleFilter originFilter, BuiltInFunctionTupleFilter function, FilterOperatorEnum op, Set<?> values, Object firstValue);
+
+    R visitAnd(LogicalTupleFilter originFilter, List<? extends TupleFilter> children, TupleFilterVisitor2Adaptor<R> adaptor);
+
+    R visitOr(LogicalTupleFilter originFilter, List<? extends TupleFilter> children, TupleFilterVisitor2Adaptor<R> adaptor);
+
+    R visitNot(LogicalTupleFilter originFilter, TupleFilter child, TupleFilterVisitor2Adaptor<R> adaptor);
+
+    /**
+     * @param originFilter ConstantTupleFilter.TRUE or ConstantTupleFilter.FALSE
+     * @return
+     */
+    R visitConstant(ConstantTupleFilter originFilter);
+
+    R visitUnsupported(TupleFilter originFilter);
+}
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor2Adaptor.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor2Adaptor.java
new file mode 100644
index 0000000000..1f5ce80805
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterVisitor2Adaptor.java
@@ -0,0 +1,116 @@
+/*
+ * 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.metadata.filter;
+
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import java.util.Set;
+
+public class TupleFilterVisitor2Adaptor<R> implements TupleFilterVisitor<R> {
+    private final TupleFilterVisitor2<R> visitor;
+
+    public TupleFilterVisitor2Adaptor(TupleFilterVisitor2<R> visitor) {
+        this.visitor = visitor;
+    }
+
+    @Override
+    public R visitCase(CaseTupleFilter filter) {
+        throw new UnsupportedOperationException("visitCase");
+    }
+
+    @Override
+    public R visitColumn(ColumnTupleFilter filter) {
+        // leaf node like column should never be visited
+        throw new UnsupportedOperationException("visitColumn");
+    }
+
+    @Override
+    public R visitCompare(CompareTupleFilter filter) {
+        TblColRef col = filter.getColumn();
+        FunctionTupleFilter function = filter.getFunction();
+        Set<?> values = filter.getValues();
+
+        // TODO consider case when filter.secondColumn != null
+
+        if (col != null && (!values.isEmpty() || filter.getOperator() == FilterOperatorEnum.ISNOTNULL || filter.getOperator() == FilterOperatorEnum.ISNULL)) {
+            return visitor.visitColumnCompare(filter, col, filter.operator, values, filter.getFirstValue());
+        }
+
+        if (function instanceof BuiltInFunctionTupleFilter) {
+            BuiltInFunctionTupleFilter functionFilter = (BuiltInFunctionTupleFilter) function;
+            if (functionFilter.getColumn() != null && !values.isEmpty()) {
+                return visitor.visitColumnFunction(filter, functionFilter, filter.operator, values, filter.getFirstValue());
+            }
+        }
+
+        // TODO consider MassInTupleFilter?
+        return visitor.visitUnsupported(filter);
+    }
+
+    @Override
+    public R visitConstant(ConstantTupleFilter filter) {
+        if (filter == ConstantTupleFilter.TRUE || filter == ConstantTupleFilter.FALSE) {
+            return visitor.visitConstant(filter);
+        }
+        throw new AssertionError("visitConstant"); // should never traverse a non-root constant filter
+    }
+
+    @Override
+    public R visitDynamic(DynamicTupleFilter filter) {
+        throw new UnsupportedOperationException("visitDynamic");
+    }
+
+    @Override
+    public R visitFunction(FunctionTupleFilter filter) {
+        if (!(filter instanceof BuiltInFunctionTupleFilter)) {
+            return visitor.visitUnsupported(filter);
+        }
+        BuiltInFunctionTupleFilter filter2 = (BuiltInFunctionTupleFilter) filter;
+        if ("LIKE".equals(filter2.name) && filter2.getColumn() != null && filter2.getConstantTupleFilter() != null) {
+            String pattern = (String) filter2.getConstantTupleFilter().getValues().iterator().next();
+            return visitor.visitColumnLike(filter2, filter2.getColumn(), pattern, filter2.isReversed());
+        }
+        return visitor.visitUnsupported(filter);
+    }
+
+    @Override
+    public R visitLogical(LogicalTupleFilter filter) {
+        if (filter.operator == FilterOperatorEnum.AND) {
+            return visitor.visitAnd(filter, filter.getChildren(), this);
+        }
+        if (filter.operator == FilterOperatorEnum.OR) {
+            return visitor.visitOr(filter, filter.getChildren(), this);
+        }
+        if (filter.operator == FilterOperatorEnum.NOT) {
+            return visitor.visitNot(filter, filter.getChildren().get(0), this);
+        }
+        throw new AssertionError("Illegal operator for LogicalTupleFilter: " + filter.operator);
+    }
+
+    @Override
+    public R visitExtract(ExtractTupleFilter filter) {
+        throw new UnsupportedOperationException("visitExtract");
+    }
+
+    @Override
+    public R visitUnsupported(UnsupportedTupleFilter filter) {
+        return visitor.visitUnsupported(filter);
+    }
+}
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilters.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilters.java
new file mode 100644
index 0000000000..50601fbb13
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilters.java
@@ -0,0 +1,148 @@
+/*
+ * 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.metadata.filter;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.base.Preconditions;
+
+public class TupleFilters {
+
+    public static TupleFilter eq(TblColRef column, Object value) {
+        return compareSingle(column, FilterOperatorEnum.EQ, value);
+    }
+
+    private static TupleFilter compareSingle(TblColRef column, FilterOperatorEnum op, Object value) {
+        CompareTupleFilter filter = new CompareTupleFilter(op);
+        filter.addChild(new ColumnTupleFilter(column));
+        filter.addChild(new ConstantTupleFilter(value));
+        return filter;
+    }
+
+    public static TupleFilter compare(TblColRef column, FilterOperatorEnum op, Collection<?> values) {
+        CompareTupleFilter filter = new CompareTupleFilter(op);
+        filter.addChild(new ColumnTupleFilter(column));
+        filter.addChild(new ConstantTupleFilter(values));
+        return filter;
+    }
+
+    public static TupleFilter like(TblColRef column, String pattern, boolean reversed) {
+        BuiltInFunctionTupleFilter filter = new BuiltInFunctionTupleFilter("LIKE");
+        filter.addChild(new ColumnTupleFilter(column));
+        filter.addChild(new ConstantTupleFilter(pattern));
+        filter.setReversed(reversed);
+        return filter;
+    }
+
+    public static TupleFilter and(List<? extends TupleFilter> children) {
+        Preconditions.checkNotNull(children);
+
+        if (children.isEmpty()) {
+            return ConstantTupleFilter.TRUE;
+        }
+
+        if (children.size() == 1) {
+            return children.get(0);
+        }
+
+        LogicalTupleFilter result = new LogicalTupleFilter(FilterOperatorEnum.AND);
+        for (TupleFilter child : children) {
+            // AND (true,..) => ignore
+            // AND (false,..) => false, short circuit
+            if (child == ConstantTupleFilter.TRUE) {
+                continue;
+            }
+            if (child == ConstantTupleFilter.FALSE) {
+                return child;
+            }
+
+            if (child.getOperator() == FilterOperatorEnum.AND) {
+                // child = and(child.getChildren()); calcite may have already flatten "and"
+                result.addChildren(child.getChildren());
+            } else {
+                result.addChild(child);
+            }
+        }
+
+        if (result.getChildren().isEmpty()) {
+            return ConstantTupleFilter.TRUE; // AND(true, true) => true
+        }
+        return result;
+    }
+
+    public static TupleFilter or(List<? extends TupleFilter> children) {
+        Preconditions.checkNotNull(children);
+
+        if (children.isEmpty()) {
+            return ConstantTupleFilter.TRUE;
+        }
+
+        if (children.size() == 1) {
+            return children.get(0);
+        }
+
+        LogicalTupleFilter result = new LogicalTupleFilter(FilterOperatorEnum.OR);
+        for (TupleFilter child : children) {
+            // OR (true,..) => true, short circuit
+            // OR (false,..) => ignore
+            if (child == ConstantTupleFilter.TRUE) {
+                return child;
+            }
+            if (child == ConstantTupleFilter.FALSE) {
+                continue;
+            }
+
+            if (child.getOperator() == FilterOperatorEnum.OR) {
+                // child = or(child.getChildren()); calcite may have already flatten "or"
+                result.addChildren(child.getChildren());
+            } else {
+                result.addChild(child);
+            }
+        }
+
+        if (result.getChildren().isEmpty()) {
+            return ConstantTupleFilter.FALSE; // OR(false, false) => false
+        }
+        return result;
+    }
+
+    public static TupleFilter not(TupleFilter filter) {
+        Preconditions.checkNotNull(filter);
+
+        if (filter == ConstantTupleFilter.TRUE) {
+            return ConstantTupleFilter.FALSE;
+        }
+
+        if (filter == ConstantTupleFilter.FALSE) {
+            return ConstantTupleFilter.TRUE;
+        }
+
+        if (filter.getOperator() == FilterOperatorEnum.NOT) {
+            return filter.getChildren().get(0);
+        }
+
+        LogicalTupleFilter notFilter = new LogicalTupleFilter(FilterOperatorEnum.NOT);
+        notFilter.addChild(filter);
+        return notFilter;
+    }
+}
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java
index 85605d4df5..a3cb309d8a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java
@@ -56,4 +56,9 @@ public void serialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
     @Override
     public void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
     }
+
+    @Override
+    public <R> R accept(TupleFilterVisitor<R> visitor) {
+        return visitor.visitUnsupported(this);
+    }
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index 72d956a359..558b90b650 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -62,6 +62,10 @@ public static FunctionDesc newInstance(String expression, ParameterDesc param, S
     public static final String FUNC_COUNT_DISTINCT = "COUNT_DISTINCT";
     public static final String FUNC_GROUPING = "GROUPING";
     public static final String FUNC_PERCENTILE = "PERCENTILE_APPROX";
+    public static final String FUNC_RAW = "RAW";
+    public static final String FUNC_TOP_N = "TOP_N";
+    public static final String DATATYPE_HLLC = "hllc";
+
     public static final Set<String> BUILT_IN_AGGREGATIONS = Sets.newHashSet();
 
     static {
@@ -201,6 +205,23 @@ public boolean isCountDistinct() {
         return FUNC_COUNT_DISTINCT.equalsIgnoreCase(expression);
     }
 
+    public boolean isHll() {
+        return FUNC_COUNT_DISTINCT.equalsIgnoreCase(expression) && DATATYPE_HLLC.equalsIgnoreCase(returnType);
+    }
+
+    public boolean isRaw() {
+        return FUNC_RAW.equalsIgnoreCase(expression);
+    }
+
+    public boolean isTopN() {
+        return FUNC_TOP_N.equalsIgnoreCase(expression);
+    }
+
+    public boolean isPercentile() {
+        return FUNC_PERCENTILE.equalsIgnoreCase(expression);
+    }
+
+
     /**
      * Get Full Expression such as sum(amount), count(1), count(*)...
      */
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java
index e55257435b..9545252261 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java
@@ -23,6 +23,7 @@
     public static final int ID_HBASE = 0;
     public static final int ID_HYBRID = 1;
     public static final int ID_SHARDED_HBASE = 2;
+    public static final int ID_SHARDED_DRUID = 5;
 
     int getStorageType();
 }
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index 86db79b0f2..fdc2f3a655 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -123,6 +123,8 @@ public void setMeasureValue(int idx, Object fieldValue) {
             }
         } else if ("float".equals(dataType) && fieldValue instanceof BigDecimal) {
             fieldValue = ((BigDecimal) fieldValue).floatValue();
+        } else if ("bigint".equals(dataType) && fieldValue instanceof Number) {
+            fieldValue = ((Number) fieldValue).longValue();
         } else if ("integer".equals(dataType) && fieldValue instanceof Number) {
             fieldValue = ((Number) fieldValue).intValue();
         } else if ("bigint".equals(dataType) && fieldValue instanceof Number) {
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
index 5455cc33f5..ca4cc5f703 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/IStorageQuery.java
@@ -28,4 +28,8 @@
 
     ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo);
 
+    /**
+     * @return whether calcite's runtime filter over scan node should be kept or not.
+     */
+    boolean keepRuntimeFilter();
 }
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index 5d2d06fb0f..6555cc518c 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -20,7 +20,9 @@
 
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.kylin.common.QueryContextFacade;
 import org.apache.kylin.common.StorageURL;
+import org.apache.kylin.common.QueryContext;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
@@ -41,6 +43,7 @@
     public final int ctxId;
 
     private StorageURL connUrl;
+    private final QueryContext queryContext;
     private int limit = Integer.MAX_VALUE;
     private boolean overlookOuterLimit = false;
     private int offset = 0;
@@ -70,6 +73,7 @@ public StorageContext() {
 
     public StorageContext(int ctxId) {
         this.ctxId = ctxId;
+        this.queryContext = QueryContextFacade.current();
     }
 
     public long getFilterMask() {
@@ -80,6 +84,10 @@ public void setFilterMask(long filterMask) {
         this.filterMask = filterMask;
     }
 
+    public QueryContext getQueryContext() {
+        return queryContext;
+    }
+
     public StorageURL getConnUrl() {
         return connUrl;
     }
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java
index 0107434118..678f390fae 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java
@@ -58,4 +58,15 @@ public ITupleIterator search(final StorageContext context, final SQLDigest sqlDi
         return new CompoundTupleIterator(tupleIterators);
     }
 
+    @Override
+    public boolean keepRuntimeFilter() {
+        boolean result = storageEngines[0].keepRuntimeFilter();
+        for (int i = 1; i < storageEngines.length; i++) {
+            if (storageEngines[i].keepRuntimeFilter() ^ result) {
+                throw new RuntimeException("inconsistent between realizations, some keep runtime filter, some don't");
+            }
+        }
+        return result;
+    }
+
 }
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
index d80df7870b..7dab2a668a 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
@@ -54,10 +54,10 @@
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.ExtractTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.filter.UnsupportedTupleFilter;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.TblColRef.InnerDataTypeEnum;
@@ -304,7 +304,7 @@ public void verifyScanWithUnevaluatableFilter() throws IOException {
         GTInfo info = table.getInfo();
 
         CompareTupleFilter fComp = compare(info.colRef(0), FilterOperatorEnum.GT, enc(info, 0, "2015-01-14"));
-        ExtractTupleFilter fUnevaluatable = unevaluatable(info.colRef(1));
+        TupleFilter fUnevaluatable = unevaluatable(info.colRef(1));
         LogicalTupleFilter fNotPlusUnevaluatable = not(unevaluatable(info.colRef(1)));
         LogicalTupleFilter filter = and(fComp, fUnevaluatable, fNotPlusUnevaluatable);
 
@@ -574,8 +574,8 @@ public static ByteArray enc(GTInfo info, int col, String value) {
         return ByteArray.copyOf(buf.array(), buf.arrayOffset(), buf.position());
     }
 
-    public static ExtractTupleFilter unevaluatable(TblColRef col) {
-        ExtractTupleFilter r = new ExtractTupleFilter(FilterOperatorEnum.EXTRACT);
+    public static TupleFilter unevaluatable(TblColRef col) {
+        UnsupportedTupleFilter r = new UnsupportedTupleFilter(FilterOperatorEnum.UNSUPPORTED);
         r.addChild(new ColumnTupleFilter(col));
         return r;
     }
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
index 28c62db3b8..575f8dc0c4 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
@@ -40,7 +40,7 @@ protected void bindCurrentConfiguration(Configuration conf) {
     }
 
     @Override
-    final protected void setup(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context context)
+    protected void setup(Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context context)
             throws IOException, InterruptedException {
         try {
             logger.info("Do setup, available memory: {}m", MemoryBudgetController.getSystemAvailMB());
diff --git a/pom.xml b/pom.xml
index 680144f3f2..ebe1e33875 100644
--- a/pom.xml
+++ b/pom.xml
@@ -56,6 +56,9 @@
     <!-- HBase versions -->
     <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
 
+    <!-- Druid versions -->
+    <druid.version>0.11.0</druid.version>
+
     <!-- Kafka versions -->
     <kafka.version>1.0.0</kafka.version>
 
@@ -117,7 +120,7 @@
     <ehcache.version>2.10.2.2.21</ehcache.version>
     <memcached.verion>2.12.3</memcached.verion>
     <apache-httpclient.version>4.2.5</apache-httpclient.version>
-    <roaring.version>0.6.18</roaring.version>
+    <roaring.version>0.5.18</roaring.version>
     <cglib.version>3.2.4</cglib.version>
     <supercsv.version>2.4.0</supercsv.version>
     <cors.version>2.5</cors.version>
@@ -295,6 +298,11 @@
         <artifactId>kylin-storage-hbase</artifactId>
         <version>${project.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.kylin</groupId>
+        <artifactId>kylin-storage-druid</artifactId>
+        <version>${project.version}</version>
+      </dependency>
       <dependency>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin-query</artifactId>
@@ -702,6 +710,66 @@
         <scope>provided</scope>
       </dependency>
 
+      <!-- Druid dependency -->
+      <dependency>
+        <groupId>io.druid</groupId>
+        <artifactId>druid-processing</artifactId>
+        <version>${druid.version}</version>
+        <exclusions>
+          <!-- exclude log4j 2 slf4j binding and impl -->
+          <exclusion>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-slf4j-impl</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-jul</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-1.2-api</artifactId>
+          </exclusion>
+          <!-- Jackson -->
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+          </exclusion>
+          <!-- unused dependencies -->
+          <exclusion>
+            <groupId>com.google.inject</groupId>
+            <artifactId>guice</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.google.inject.extensions</groupId>
+            <artifactId>guice-multibindings</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.google.protobuf</groupId>
+            <artifactId>protobuf-java</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.mozilla</groupId>
+            <artifactId>rhino</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+
       <!-- Other dependencies -->
       <dependency>
         <groupId>org.apache.commons</groupId>
@@ -1277,6 +1345,7 @@
     <module>source-jdbc</module>
     <module>source-kafka</module>
     <module>storage-hbase</module>
+    <module>storage-druid</module>
     <module>query</module>
     <module>server-base</module>
     <module>server</module>
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index b34b42e135..db9974a8bc 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -47,13 +47,16 @@
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+import org.apache.kylin.storage.IStorageQuery;
+import org.apache.kylin.storage.StorageFactory;
 
 /**
  */
 public class OLAPFilterRel extends Filter implements OLAPRel {
-
-    ColumnRowType columnRowType;
-    OLAPContext context;
+    private ColumnRowType columnRowType;
+    private OLAPContext context;
+    private boolean afterAggregate;
+    private boolean hasRuntimeFilter = true;
 
     public OLAPFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
         super(cluster, traits, child, condition);
@@ -79,9 +82,10 @@ public void implementOLAP(OLAPImplementor implementor) {
 
         this.columnRowType = buildColumnRowType();
         this.context = implementor.getContext();
+        this.afterAggregate = context.afterAggregate;
 
         // only translate where clause and don't translate having clause
-        if (!context.afterAggregate) {
+        if (!afterAggregate) {
             translateFilter(context);
         } else {
             context.afterHavingClauseFilter = true;
@@ -146,7 +150,16 @@ private TupleFilter and(TupleFilter f1, TupleFilter f2) {
 
     @Override
     public EnumerableRel implementEnumerable(List<EnumerableRel> inputs) {
-        // keep it for having clause
+        EnumerableRel input = sole(inputs);
+        final boolean overCube = !afterAggregate && context.realization != null &&  context.realization.getModel().isFactTable(context.firstTableScan.getTableName());
+        if (overCube) {
+            IStorageQuery storageQuery = StorageFactory.createQuery(context.realization);
+            if (!storageQuery.keepRuntimeFilter()) {
+                hasRuntimeFilter = false;
+                return input;
+            }
+        }
+        // keep it for all other cases like having clause
         RexBuilder rexBuilder = getCluster().getRexBuilder();
         RelDataType inputRowType = getInput().getRowType();
         RexProgramBuilder programBuilder = new RexProgramBuilder(inputRowType, rexBuilder);
@@ -194,4 +207,8 @@ public RelWriter explainTerms(RelWriter pw) {
         return super.explainTerms(pw).item("ctx",
                 context == null ? "" : String.valueOf(context.id) + "@" + context.realization);
     }
+
+    public boolean hasRuntimeFilter() {
+        return hasRuntimeFilter;
+    }
 }
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
index de4b43892f..7100df02c4 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
@@ -201,11 +201,17 @@ private TblColRef translateRexNode(TupleExpression tupleExpr, String fieldName)
     @Override
     public EnumerableRel implementEnumerable(List<EnumerableRel> inputs) {
         if (getInput() instanceof OLAPFilterRel) {
-            // merge project & filter
             OLAPFilterRel filter = (OLAPFilterRel) getInput();
-            RelNode inputOfFilter = inputs.get(0).getInput(0);
+            RexNode condition = null;
+            RelNode inputOfFilter = inputs.get(0);
+
+            if (filter.hasRuntimeFilter()) { // merge project & filter
+                condition = filter.getCondition();
+                inputOfFilter = inputOfFilter.getInput(0);
+            }
+
             RexProgram program = RexProgram.create(inputOfFilter.getRowType(), this.rewriteProjects,
-                    filter.getCondition(), this.rowType, getCluster().getRexBuilder());
+                    condition, this.rowType, getCluster().getRexBuilder());
             return new EnumerableCalc(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), //
                     inputOfFilter, program);
         } else {
diff --git a/server-base/pom.xml b/server-base/pom.xml
index 4cd3f76247..264cc56519 100644
--- a/server-base/pom.xml
+++ b/server-base/pom.xml
@@ -56,6 +56,10 @@
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-metrics-reporter-hive</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-storage-druid</artifactId>
+        </dependency>
         <dependency>
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-metrics-reporter-kafka</artifactId>
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index b76f7b9268..4937acdb98 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -51,6 +51,9 @@
 import org.apache.kylin.metadata.model.ISourceAware;
 import org.apache.kylin.metadata.model.SegmentRange;
 import org.apache.kylin.metadata.model.SegmentRange.TSRange;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.IStorageAware;
+import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.metrics.MetricsManager;
@@ -589,6 +592,8 @@ public CubeRequest saveCubeDesc(@RequestBody CubeRequest cubeRequest) {
             throw new BadRequestException("Invalid Cube name, only letters, numbers and underscore supported.");
         }
 
+        validateStorageType(desc);
+
         try {
             desc.setUuid(RandomUtil.randomUUID().toString());
             String projectName = (null == cubeRequest.getProject()) ? ProjectInstance.DEFAULT_PROJECT_NAME
@@ -608,6 +613,18 @@ public CubeRequest saveCubeDesc(@RequestBody CubeRequest cubeRequest) {
         return cubeRequest;
     }
 
+    private void validateStorageType(CubeDesc cubeDesc) {
+        if (cubeDesc.getStorageType() == IStorageAware.ID_SHARDED_DRUID) {
+            List<MeasureDesc> measureDescs = cubeDesc.getMeasures();
+            for(MeasureDesc measureDesc: measureDescs) {
+                FunctionDesc functionDesc = measureDesc.getFunction();
+                if (functionDesc.isHll() || functionDesc.isRaw() || functionDesc.isTopN() || functionDesc.isPercentile()) {
+                    throw new BadRequestException("Druid Storage Engine don't support HLL, Raw, TopN, Percentile measure, please choose HBase");
+                }
+            }
+        }
+    }
+
     /**
      * update CubDesc
      *
@@ -635,6 +652,8 @@ public CubeRequest updateCubeDesc(@RequestBody CubeRequest cubeRequest) throws J
                 return cubeRequest;
             }
 
+            validateStorageType(desc);
+
             //cube renaming is not allowed
             if (!cube.getDescriptor().getName().equalsIgnoreCase(desc.getName())) {
                 String error = "Cube Desc renaming is not allowed: desc.getName(): " + desc.getName()
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 96d60c71ff..9135e57129 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -32,6 +32,9 @@
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.common.util.CliCommandExecutor;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.kylin.storage.druid.common.DruidCoordinatorClient;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -80,6 +83,7 @@
 import org.apache.kylin.rest.util.AclEvaluate;
 import org.apache.kylin.rest.util.ValidateUtil;
 import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.apache.kylin.storage.druid.DruidSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.InitializingBean;
@@ -213,6 +217,8 @@ public CubeInstance createCubeAndDesc(ProjectInstance project, CubeDesc desc) th
         CubeDesc createdDesc;
         CubeInstance createdCube;
 
+        setOnlineVersionForDruidStorage(desc, null);
+
         createdDesc = getCubeDescManager().createCubeDesc(desc);
 
         if (createdDesc.isBroken()) {
@@ -226,6 +232,23 @@ public CubeInstance createCubeAndDesc(ProjectInstance project, CubeDesc desc) th
         return createdCube;
     }
 
+
+    private void setOnlineVersionForDruidStorage(CubeDesc cubeDesc, CubeInstance cubeInstance) {
+        if (cubeDesc.getOnlineVersion() == 0 && cubeDesc.getStorageType() == 5 && (cubeInstance == null || cubeInstance.getSegments().size() == 0)) {
+            List<String> datasources = DruidCoordinatorClient.getSingleton().getDataSource();
+            int version = -1;
+            String regex = cubeDesc.getName() + "_(\\d*)";
+            Pattern pattern = Pattern.compile(regex);
+            for(String datasource: datasources) {
+                Matcher matcher = pattern.matcher(datasource);
+                if (matcher.matches()) {
+                    version = Math.max(version, Integer.parseInt(matcher.group(matcher.groupCount())));
+                }
+            }
+            cubeDesc.setOnlineVersion(++version);
+        }
+    }
+
     public List<CubeInstance> listAllCubes(String projectName) {
         ProjectManager projectManager = getProjectManager();
         ProjectInstance project = projectManager.getProject(projectName);
@@ -277,16 +300,19 @@ public CubeDesc updateCubeAndDesc(CubeInstance cube, CubeDesc desc, String newPr
         if (!cubingJobs.isEmpty()) {
             throw new BadRequestException(String.format(Locale.ROOT, msg.getDISCARD_JOB_FIRST(), cube.getName()));
         }
-
+        
         //double check again
         if (!forceUpdate && !cube.getDescriptor().consistentWith(desc)) {
             throw new BadRequestException(String.format(Locale.ROOT, msg.getINCONSISTENT_CUBE_DESC(), desc.getName()));
         }
 
+        setOnlineVersionForDruidStorage(desc, cube);
+
         CubeDesc updatedCubeDesc = getCubeDescManager().updateCubeDesc(desc);
         int cuboidCount = CuboidCLI.simulateCuboidGeneration(updatedCubeDesc, false);
         logger.info("Updated cube " + cube.getName() + " has " + cuboidCount + " cuboids");
 
+
         ProjectManager projectManager = getProjectManager();
         if (!isCubeInProject(newProjectName, cube)) {
             String owner = SecurityContextHolder.getContext().getAuthentication().getName();
@@ -308,6 +334,12 @@ public void deleteCube(CubeInstance cube) throws IOException {
         }
 
         try {
+            //for Druid Storage
+            if (cube.getStorageType() == 5 && cube.getSegments().size() > 0) {
+                String dataSource = DruidSchema.getDataSource(cube.getDescriptor());
+                DruidCoordinatorClient.getSingleton().deleteDataSource(dataSource);
+            }
+
             this.releaseAllJobs(cube);
         } catch (Exception e) {
             logger.error("error when releasing all jobs", e);
@@ -373,9 +405,18 @@ public CubeInstance purgeCube(CubeInstance cube) throws IOException {
                     String.format(Locale.ROOT, msg.getPURGE_NOT_DISABLED_CUBE(), cubeName, ostatus));
         }
 
-        this.releaseAllSegments(cube);
-        return cube;
+        try {
+            //for Druid Storage
+            if (cube.getStorageType() == 5 && cube.getSegments().size() > 0) {
+                String dataSource = DruidSchema.getDataSource(cube.getDescriptor());
+                DruidCoordinatorClient.getSingleton().deleteDataSource(dataSource);
+            }
 
+            this.releaseAllSegments(cube);
+            return cube;
+        } catch (IOException e) {
+            throw e;
+        }
     }
 
     /**
diff --git a/server/src/test/java/org/apache/kylin/rest/service/KylinQueryTimeoutTest.java b/server/src/test/java/org/apache/kylin/rest/service/KylinQueryTimeoutTest.java
index f2cf276a1d..00f71119f5 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/KylinQueryTimeoutTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/KylinQueryTimeoutTest.java
@@ -105,5 +105,10 @@ public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleI
             }
             return null;
         }
+
+        @Override
+        public boolean keepRuntimeFilter() {
+            return true;
+        }
     }
 }
\ No newline at end of file
diff --git a/storage-druid/README.md b/storage-druid/README.md
new file mode 100644
index 0000000000..d60fce80af
--- /dev/null
+++ b/storage-druid/README.md
@@ -0,0 +1,40 @@
+### How to compile
+
+It's the same as Kylin on HBase
+
+### How to deploy
+
+It's the same as Kylin on HBase
+
+### The config for Kylin on Druid
+
+```
+#support multi coordinators failover
+kylin.storage.druid.coordinator-addresses=http://host1:port,http://host2:port
+
+kylin.storage.druid.broker-host=host:port
+
+#the druid segments path in HDFS
+kylin.storage.druid.hdfs-location=/druid/segments
+
+#the mysql metastore info for druid, must same as Druid config
+kylin.storage.druid.mysql-url=jdbc:mysql://host:port/db?characterEncoding=UTF-8
+kylin.storage.druid.mysql-user=xxx
+kylin.storage.druid.mysql-password=xxx
+kylin.storage.druid.mysql-seg-table=segments
+
+kylin.storage.druid.max-shard-count=500
+
+#avoid dependency conflicts
+kylin.engine.mr.druid-config-override.mapreduce.job.user.classpath.first=true
+
+#Druid need JDK8
+kylin.engine.mr.druid-config-override.yarn.app.mapreduce.am.env=JAVA_HOME=/usr/local/java18
+kylin.engine.mr.druid-config-override.mapred.child.env=JAVA_HOME=/usr/local/java18
+
+kylin.engine.mr.druid-config-override.mapreduce.reduce.memory.mb=3500
+kylin.engine.mr.druid-config-override.mapreduce.reduce.speculative=false
+```
+
+### How to use Kylin Druid Storage
+when create or update cube, change the "Cube Storage" to Druid in "Advanced Setting"
\ No newline at end of file
diff --git a/storage-druid/pom.xml b/storage-druid/pom.xml
new file mode 100644
index 0000000000..6bab5fd32b
--- /dev/null
+++ b/storage-druid/pom.xml
@@ -0,0 +1,72 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kylin-storage-druid</artifactId>
+    <packaging>jar</packaging>
+    <name>Apache Kylin - Druid Storage</name>
+    <description>Apache Kylin - Druid Storage</description>
+
+    <parent>
+        <artifactId>kylin</artifactId>
+        <groupId>org.apache.kylin</groupId>
+        <version>2.6.0-SNAPSHOT</version>
+    </parent>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-engine-mr</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>io.druid</groupId>
+            <artifactId>druid-processing</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-app</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>mysql</groupId>
+            <artifactId>mysql-connector-java</artifactId>
+            <version>5.1.40</version>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/DruidSchema.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/DruidSchema.java
new file mode 100644
index 0000000000..8f473c4bdb
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/DruidSchema.java
@@ -0,0 +1,236 @@
+/*
+ * 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.storage.druid;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.druid.java.util.common.Intervals;
+import io.druid.query.ordering.StringComparator;
+import io.druid.query.ordering.StringComparators;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeDesc;
+//import org.apache.kylin.measure.bitmap.BitmapMeasureType;
+//import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Interval;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import io.druid.data.input.impl.DimensionSchema;
+import io.druid.data.input.impl.StringDimensionSchema;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
+import io.druid.query.aggregation.DoubleMinAggregatorFactory;
+import io.druid.query.aggregation.DoubleSumAggregatorFactory;
+import io.druid.query.aggregation.FloatMaxAggregatorFactory;
+import io.druid.query.aggregation.FloatMinAggregatorFactory;
+import io.druid.query.aggregation.FloatSumAggregatorFactory;
+import io.druid.query.aggregation.LongMaxAggregatorFactory;
+import io.druid.query.aggregation.LongMinAggregatorFactory;
+import io.druid.query.aggregation.LongSumAggregatorFactory;
+//import io.druid.query.aggregation.decimal.DecimalMaxAggregatorFactory;
+//import io.druid.query.aggregation.decimal.DecimalMinAggregatorFactory;
+//import io.druid.query.aggregation.decimal.DecimalSumAggregatorFactory;
+//import io.druid.query.aggregation.kylin.distinctcount.DistinctCountAggregatorFactory;
+//import io.druid.query.aggregation.kylin.extendcolumn.ExtendColumnAggregatorFactory;
+import io.druid.query.dimension.DefaultDimensionSpec;
+import io.druid.query.dimension.DimensionSpec;
+
+public class DruidSchema implements NameMapping {
+    public static final String ID_COL = "_CUBOID_ID_";
+
+    public static final Interval ETERNITY_INTERVAL = new Interval(
+            new DateTime("0000-01-01", DateTimeZone.UTC),
+            new DateTime("3000-01-01", DateTimeZone.UTC)
+    );
+
+    private final NameMapping mapping;
+    private final List<TblColRef> dimensions;
+    private final List<MeasureDesc> measures;
+
+    // computed
+    private Map<String, Integer> indexByName = new HashMap<>();
+
+    public DruidSchema(NameMapping mapping, List<TblColRef> dimensions, List<MeasureDesc> measures) {
+        this.mapping = mapping;
+        this.dimensions = ImmutableList.copyOf(dimensions);
+        this.measures = ImmutableList.copyOf(measures);
+
+        int fieldIndex = 0;
+        for (TblColRef dim : dimensions) {
+            String fieldName = getDimFieldName(dim);
+            indexByName.put(fieldName, fieldIndex++);
+        }
+        for (MeasureDesc measure : measures) {
+            String fieldName = getMeasureFieldName(measure);
+            indexByName.put(fieldName, fieldIndex++);
+        }
+    }
+
+    public static Interval segmentInterval(CubeSegment segment) {
+        if (!segment.getCubeDesc().getModel().getPartitionDesc().isPartitioned()) {
+            return Intervals.utc(0, segment.getCreateTimeUTC());
+        }
+        return Intervals.utc(segment.getTSRange().start.v, segment.getTSRange().end.v);
+    }
+
+    public static String getDataSource(CubeDesc cubeDesc) {
+        return cubeDesc.getName() + "_" + cubeDesc.getOnlineVersion();
+    }
+
+    public static StringComparator dimensionComparator(DataType dataType) {
+        if (dataType.isStringFamily()) {
+            return StringComparators.LEXICOGRAPHIC;
+        }
+        return StringComparators.NUMERIC;
+    }
+
+    @Override
+    public String getDimFieldName(TblColRef dim) {
+        return mapping.getDimFieldName(dim);
+    }
+
+    @Override
+    public String getMeasureFieldName(MeasureDesc measure) {
+        return mapping.getMeasureFieldName(measure);
+    }
+
+    public List<TblColRef> getDimensions() {
+        return dimensions;
+    }
+
+    public List<MeasureDesc> getMeasures() {
+        return measures;
+    }
+
+    public int getTotalFieldCount() {
+        return dimensions.size() + measures.size();
+    }
+
+    public int getFieldIndex(String field) {
+        return indexByName.get(field);
+    }
+
+    public List<DimensionSchema> getDimensionSchemas() {
+        List<DimensionSchema> result = new ArrayList<>();
+        result.add(new StringDimensionSchema(ID_COL));
+        for (TblColRef dim : dimensions) {
+            result.add(new StringDimensionSchema(getDimFieldName(dim)));
+        }
+        return result;
+    }
+
+    public List<DimensionSpec> getQueryDimensionSpec(Collection<TblColRef> dimensions) {
+        List<DimensionSpec> result = Lists.newArrayListWithCapacity(dimensions.size());
+        for (TblColRef dim : dimensions) {
+            String name = getDimFieldName(dim);
+            result.add(new DefaultDimensionSpec(name, name));
+        }
+        return result;
+    }
+
+    public AggregatorFactory[] getAggregators() {
+        return getAggregators(measures);
+    }
+
+    public AggregatorFactory[] getAggregators(List<MeasureDesc> measures) {
+        Iterable<AggregatorFactory> aggregatorFactories = Iterables.transform(measures, new Function<MeasureDesc, AggregatorFactory>() {
+            @Override
+            public AggregatorFactory apply(MeasureDesc measure) {
+                String name = getMeasureFieldName(measure);
+                FunctionDesc func = measure.getFunction();
+                DataType returnType = func.getReturnDataType();
+                switch (func.getExpression()) {
+                case FunctionDesc.FUNC_COUNT:
+                    return new LongSumAggregatorFactory(name, name);
+                case FunctionDesc.FUNC_SUM: {
+                    if (returnType.isIntegerFamily()) {
+                        return new LongSumAggregatorFactory(name, name);
+                    }
+                    if (returnType.isFloat()) {
+                        return new FloatSumAggregatorFactory(name, name);
+                    }
+                    if (returnType.isDouble()) {
+                        return new DoubleSumAggregatorFactory(name, name);
+                    }
+//                    if (returnType.isDecimal()) {
+//                        return new DecimalSumAggregatorFactory(name, name, returnType.getPrecision());
+//                    }
+                    break;
+                }
+                case FunctionDesc.FUNC_MIN: {
+                    if (returnType.isIntegerFamily()) {
+                        return new LongMinAggregatorFactory(name, name);
+                    }
+                    if (returnType.isFloat()) {
+                        return new FloatMinAggregatorFactory(name, name);
+                    }
+                    if (returnType.isDouble()) {
+                        return new DoubleMinAggregatorFactory(name, name);
+                    }
+//                    if (returnType.isDecimal()) {
+//                        return new DecimalMinAggregatorFactory(name, name, returnType.getPrecision());
+//                    }
+                    break;
+                }
+                case FunctionDesc.FUNC_MAX: {
+                    if (returnType.isIntegerFamily()) {
+                        return new LongMaxAggregatorFactory(name, name);
+                    }
+                    if (returnType.isFloat()) {
+                        return new FloatMaxAggregatorFactory(name, name);
+                    }
+                    if (returnType.isDouble()) {
+                        return new DoubleMaxAggregatorFactory(name, name);
+                    }
+//                    if (returnType.isDecimal()) {
+//                        return new DecimalMaxAggregatorFactory(name, name, returnType.getPrecision());
+//                    }
+                    break;
+                }
+//                case FunctionDesc.FUNC_COUNT_DISTINCT:
+//                    if (returnType.getName().equals(BitmapMeasureType.DATATYPE_BITMAP)) {
+//                        return new DistinctCountAggregatorFactory(name, name);
+//                    }
+//                    break;
+//                case ExtendedColumnMeasureType.FUNC_EXTENDED_COLUMN:
+//                    return new ExtendColumnAggregatorFactory(name, name, returnType.getPrecision());
+                    // TODO support hll
+                default:
+                    throw new AssertionError(func.getExpression());
+                }
+                throw new UnsupportedOperationException(func + " is not supported");
+
+            }
+        });
+        return Iterables.toArray(aggregatorFactories, AggregatorFactory.class);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/DruidStorage.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/DruidStorage.java
new file mode 100644
index 0000000000..7f53358483
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/DruidStorage.java
@@ -0,0 +1,68 @@
+/*
+ * 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.storage.druid;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.engine.mr.IMROutput2;
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.storage.IStorage;
+import org.apache.kylin.storage.IStorageQuery;
+import org.apache.kylin.storage.druid.common.DruidSerdeHelper;
+import org.apache.kylin.storage.druid.http.HttpClient;
+import org.apache.kylin.storage.druid.http.HttpClientConfig;
+import org.apache.kylin.storage.druid.http.HttpClientFactory;
+import org.apache.kylin.storage.druid.read.DruidStorageQuery;
+import org.apache.kylin.storage.druid.read.cursor.RowCursorFactory;
+import org.apache.kylin.storage.druid.write.DruidMROutput;
+import org.joda.time.Duration;
+
+public class DruidStorage implements IStorage {
+    private final HttpClient httpClient;
+    private final RowCursorFactory cursorFactory;
+
+    public DruidStorage() {
+        final HttpClientConfig.Builder builder = HttpClientConfig
+                .builder()
+                .withNumConnections(100)
+                .withReadTimeout(Duration.standardSeconds(60))
+                .withWorkerCount(40)
+                .withCompressionCodec(HttpClientConfig.CompressionCodec.IDENTITY);
+
+        this.httpClient = HttpClientFactory.createClient(builder.build());
+        this.cursorFactory = new RowCursorFactory(httpClient, DruidSerdeHelper.JSON_MAPPER);
+    }
+
+    @Override
+    public IStorageQuery createQuery(IRealization realization) {
+        if (realization.getType() == RealizationType.CUBE) {
+            CubeInstance cube = (CubeInstance) realization;
+            return new DruidStorageQuery(cube, cursorFactory);
+        }
+        throw new RuntimeException("Druid storage doesn't support realization type: " + realization.getType());
+    }
+
+    @Override
+    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
+        if (engineInterface == IMROutput2.class) {
+            return engineInterface.cast(new DruidMROutput());
+        }
+        throw new RuntimeException("Druid storage can't adapt to " + engineInterface);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/NameMapping.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/NameMapping.java
new file mode 100644
index 0000000000..c2b48fb5e0
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/NameMapping.java
@@ -0,0 +1,28 @@
+/*
+ * 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.storage.druid;
+
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+public interface NameMapping {
+    String getDimFieldName(TblColRef dim);
+
+    String getMeasureFieldName(MeasureDesc met);
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/NameMappingFactory.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/NameMappingFactory.java
new file mode 100644
index 0000000000..b4dbbfb573
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/NameMappingFactory.java
@@ -0,0 +1,90 @@
+/*
+ * 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.storage.druid;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.RowKeyColDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.collect.Sets;
+
+public class NameMappingFactory {
+
+    public static NameMapping getDefault(CubeDesc desc) {
+        boolean noDupName = true;
+        Set<String> names = Sets.newHashSet();
+
+        for (RowKeyColDesc col : desc.getRowkey().getRowKeyColumns()) {
+            noDupName = noDupName && names.add(col.getColRef().getName());
+        }
+        for (MeasureDesc met : desc.getMeasures()) {
+            noDupName = noDupName && names.add(met.getName());
+        }
+
+        return noDupName ? DirectNameMapping.INSTANCE : new CachedNameMapping();
+    }
+
+    private static class DirectNameMapping implements NameMapping {
+        static DirectNameMapping INSTANCE = new DirectNameMapping();
+
+        @Override
+        public String getDimFieldName(TblColRef dim) {
+            return dim.getName();
+        }
+
+        @Override
+        public String getMeasureFieldName(MeasureDesc met) {
+            return met.getName();
+        }
+    }
+
+    private static class CachedNameMapping implements NameMapping {
+        static final String DIM_PREFIX = "D_";
+        static final String MEASURE_PREFIX = "M_";
+
+        final Map<TblColRef, String> dimNameCache = new HashMap<>();
+        final Map<String, String> metNameCache = new HashMap<>();
+
+        @Override
+        public String getDimFieldName(TblColRef dim) {
+            String name = dimNameCache.get(dim);
+            if (name == null) {
+                // prefer "@" to "." in column name
+                name = DIM_PREFIX + dim.getTableAlias() + "@" + dim.getName();
+                dimNameCache.put(dim, name);
+            }
+            return name;
+        }
+
+        @Override
+        public String getMeasureFieldName(MeasureDesc met) {
+            String name = metNameCache.get(met.getName());
+            if (name == null) {
+                name = MEASURE_PREFIX + met.getName();
+                metNameCache.put(met.getName(), name);
+            }
+            return name;
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidCoordinatorClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidCoordinatorClient.java
new file mode 100644
index 0000000000..244365e102
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidCoordinatorClient.java
@@ -0,0 +1,146 @@
+/*
+ * 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.storage.druid.common;
+
+import static org.apache.kylin.storage.druid.common.DruidSerdeHelper.JSON_MAPPER;
+
+import java.util.List;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.storage.druid.http.FullResponseHolder;
+import org.apache.kylin.storage.druid.http.HttpClient;
+import org.apache.kylin.storage.druid.http.HttpClientConfig;
+import org.apache.kylin.storage.druid.http.HttpClientFactory;
+import org.apache.kylin.storage.druid.http.Request;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.joda.time.Duration;
+import org.joda.time.Interval;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Throwables;
+
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.StringUtils;
+
+public class DruidCoordinatorClient {
+    private static final String PATH_PREFIX = "/druid/coordinator/v1";
+    private static final DruidCoordinatorClient SINGLETON = new DruidCoordinatorClient();
+
+    private DruidLeaderClient leaderClient;
+
+    private DruidCoordinatorClient() {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        HttpClientConfig.Builder builder = HttpClientConfig.builder().withNumConnections(10)
+                .withReadTimeout(Duration.standardSeconds(30)).withWorkerCount(10)
+                .withCompressionCodec(HttpClientConfig.CompressionCodec.GZIP);
+        HttpClient httpClient = HttpClientFactory.createClient(builder.build());
+        this.leaderClient = new DruidLeaderClient(httpClient, config.getDruidCoordinatorAddresses(), 10);
+    }
+
+    public static DruidCoordinatorClient getSingleton() {
+        return SINGLETON;
+    }
+
+    public List<ImmutableSegmentLoadInfo> fetchServerView(String dataSource, Interval interval) {
+        try {
+            FullResponseHolder response = leaderClient.go(leaderClient.makeRequest(HttpMethod.GET,
+                    StringUtils.format("%s/datasources/%s/intervals/%s/serverview?partial=true", PATH_PREFIX,
+                            dataSource, interval.toString().replace("/", "_"))));
+
+            if (!response.getStatus().equals(HttpResponseStatus.OK)) {
+                throw new ISE("Error while fetching serverView status[%s] content[%s]", response.getStatus(),
+                        response.getContent());
+            }
+            return JSON_MAPPER.readValue(response.getContent(), new TypeReference<List<ImmutableSegmentLoadInfo>>() {
+            });
+
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+
+    public List<Rule> getRules(String dataSource) {
+        try {
+            FullResponseHolder response = leaderClient.go(leaderClient.makeRequest(HttpMethod.GET,
+                    StringUtils.format("%s/rules/%s", PATH_PREFIX, dataSource)));
+
+            if (!response.getStatus().equals(HttpResponseStatus.OK)) {
+                throw new ISE("ERROR while fetching rules for %s: status[%s] content[%s]", dataSource,
+                        response.getStatus(), response.getContent());
+            }
+            return JSON_MAPPER.readValue(response.getContent(), new TypeReference<List<Rule>>() {
+            });
+
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+
+    public void putRules(String dataSource, List<Rule> rules) {
+        try {
+            Request request = leaderClient
+                    .makeRequest(HttpMethod.POST, StringUtils.format("%s/rules/%s", PATH_PREFIX, dataSource))
+                    .setHeader("X-Druid-Author", "Kylin-ADMIN")
+                    .setContent(MediaType.APPLICATION_JSON, JSON_MAPPER.writeValueAsBytes(rules));
+            FullResponseHolder response = leaderClient.go(request);
+
+            if (!response.getStatus().equals(HttpResponseStatus.OK)) {
+                throw new ISE("ERROR while setting rules for %s: status[%s] content[%s]", dataSource,
+                        response.getStatus(), response.getContent());
+            }
+
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+
+    public void deleteDataSource(String dataSource) {
+        try {
+            FullResponseHolder response = leaderClient.go(leaderClient.makeRequest(HttpMethod.DELETE,
+                    StringUtils.format("%s/datasources/%s", PATH_PREFIX, dataSource)));
+
+            if (!response.getStatus().equals(HttpResponseStatus.OK)) {
+                throw new ISE("ERROR while deleting datasource [%s]: status[%s] content[%s]", dataSource,
+                        response.getStatus(), response.getContent());
+            }
+
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+
+    public  List<String> getDataSource() {
+        try {
+            FullResponseHolder response = leaderClient.go(leaderClient.makeRequest(HttpMethod.GET,
+                    StringUtils.format("%s/datasources", PATH_PREFIX)));
+
+            if (!response.getStatus().equals(HttpResponseStatus.OK)) {
+                throw new ISE("ERROR while get all datasources: status[%s] content[%s]",
+                        response.getStatus(), response.getContent());
+            }
+            return JSON_MAPPER.readValue(response.getContent(), new TypeReference<List<String>>() {
+            });
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidLeaderClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidLeaderClient.java
new file mode 100644
index 0000000000..10555153a5
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidLeaderClient.java
@@ -0,0 +1,167 @@
+/*
+ * 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.storage.druid.common;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.kylin.storage.druid.http.FullResponseHandler;
+import org.apache.kylin.storage.druid.http.FullResponseHolder;
+import org.apache.kylin.storage.druid.http.HttpClient;
+import org.apache.kylin.storage.druid.http.Request;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+
+import io.druid.java.util.common.IOE;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.RE;
+import io.druid.java.util.common.StringUtils;
+
+public class DruidLeaderClient {
+    private static final Logger logger = LoggerFactory.getLogger(DruidLeaderClient.class);
+
+    private final HttpClient httpClient;
+    private final CopyOnWriteArraySet<String> hosts;
+    private final int maxRetries;
+
+    private final AtomicReference<String> currentKnownLeader;
+
+    public DruidLeaderClient(HttpClient httpClient, String[] hosts, int maxRetries) {
+        Preconditions.checkNotNull(httpClient, "httpClient");
+        Preconditions.checkNotNull(hosts, "hosts");
+        Preconditions.checkArgument(hosts.length > 0, "");
+
+        this.httpClient = httpClient;
+        this.hosts = new CopyOnWriteArraySet<>(Arrays.asList(hosts));
+        this.maxRetries = maxRetries;
+        this.currentKnownLeader = new AtomicReference<>(hosts[0]);
+    }
+
+    /**
+     * Make a Request object aimed at the leader. Throws IOException if the leader cannot be located.
+     */
+    public Request makeRequest(HttpMethod httpMethod, String urlPath) throws IOException {
+        return new Request(httpMethod, new URL(StringUtils.format("%s%s", currentKnownLeader.get(), urlPath)));
+    }
+
+    /**
+     * Executes a Request object aimed at the leader. Throws IOException if the leader cannot be located.
+     */
+    public FullResponseHolder go(Request request) throws IOException, InterruptedException {
+        String leader = currentKnownLeader.get();
+
+        for (int counter = 0; counter < maxRetries; counter++) {
+            final FullResponseHolder fullResponseHolder;
+
+            // execute, pick another node and retry if error
+            try {
+                try {
+                    fullResponseHolder = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8)).get();
+                } catch (ExecutionException e) {
+                    // Unwrap IOExceptions and ChannelExceptions, re-throw others
+                    Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
+                    Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class);
+                    throw new RE(e, "HTTP request to[%s] failed", request.getUrl());
+                }
+            } catch (IOException | ChannelException ex) {
+                logger.info("Request[{}] failed with msg [{}]. Retry.", request.getUrl(), ex.getMessage());
+                leader = getNextHost(leader);
+                try {
+                    if (request.getUrl().getQuery() == null) {
+                        request = withUrl(request,
+                                new URL(StringUtils.format("%s%s", leader, request.getUrl().getPath())));
+                    } else {
+                        request = withUrl(request, new URL(StringUtils.format("%s%s?%s", leader,
+                                request.getUrl().getPath(), request.getUrl().getQuery())));
+                    }
+                    TimeUnit.SECONDS.sleep(1);
+                    continue;
+                } catch (MalformedURLException e) {
+                    // Not an IOException; this is our own fault.
+                    throw new ISE(e, "failed to build url with path[%] and query string [%s].",
+                            request.getUrl().getPath(), request.getUrl().getQuery());
+                }
+            }
+
+            if (HttpResponseStatus.TEMPORARY_REDIRECT.equals(fullResponseHolder.getStatus())) {
+                // redirect to new leader
+                String redirectUrlStr = fullResponseHolder.getResponse().getHeader("Location");
+                if (redirectUrlStr == null) {
+                    throw new IOE("No redirect location is found in response from url[%s].", request.getUrl());
+                }
+
+                logger.info("Request[{}] received redirect response to location [{}].", request.getUrl(),
+                        redirectUrlStr);
+
+                final URL redirectUrl;
+                try {
+                    redirectUrl = new URL(redirectUrlStr);
+                } catch (MalformedURLException ex) {
+                    throw new IOE(ex,
+                            "Malformed redirect location is found in response from url[%s], new location[%s].",
+                            request.getUrl(), redirectUrlStr);
+                }
+
+                //update known leader location
+                leader = StringUtils.format("%s://%s:%s", redirectUrl.getProtocol(), redirectUrl.getHost(),
+                        redirectUrl.getPort());
+                if (hosts.add(leader)) {
+                    logger.info("Found a new leader at {}", leader);
+                }
+                request = withUrl(request, redirectUrl);
+
+            } else {
+                // success, record leader index
+                currentKnownLeader.set(leader);
+                return fullResponseHolder;
+            }
+        }
+        throw new IOE("Retries exhausted, couldn't fulfill request to [%s].", request.getUrl());
+    }
+
+    private String getNextHost(String currentHost) {
+        List<String> allHosts = Lists.newArrayList(hosts);
+        int index = allHosts.indexOf(currentHost);
+        return allHosts.get((index + 1) % allHosts.size());
+    }
+
+    private Request withUrl(Request old, URL url) {
+        Request req = new Request(old.getMethod(), url);
+        req.addHeaderValues(old.getHeaders());
+        if (old.hasContent()) {
+            req.setContent(old.getContent());
+        }
+        return req;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidSerdeHelper.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidSerdeHelper.java
new file mode 100644
index 0000000000..330901bdca
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidSerdeHelper.java
@@ -0,0 +1,79 @@
+/*
+ * 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.storage.druid.common;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import io.druid.initialization.DruidModule;
+import io.druid.jackson.DefaultObjectMapper;
+//import io.druid.query.aggregation.decimal.DecimalDruidModule;
+//import io.druid.query.aggregation.decimal.DecimalMaxSerde;
+//import io.druid.query.aggregation.decimal.DecimalMinSerde;
+//import io.druid.query.aggregation.decimal.DecimalSumSerde;
+//import io.druid.query.aggregation.kylin.distinctcount.DistinctCountDruidModule;
+//import io.druid.query.aggregation.kylin.distinctcount.DistinctCountSerde;
+//import io.druid.query.aggregation.kylin.extendcolumn.ExtendColumnAggregatorFactory;
+//import io.druid.query.aggregation.kylin.extendcolumn.ExtendColumnDruidModule;
+//import io.druid.query.aggregation.kylin.extendcolumn.ExtendColumnSerde;
+//import io.druid.segment.serde.ComplexMetrics;
+
+public class DruidSerdeHelper {
+    private static final DruidModule[] druidModules = {
+//            new DistinctCountDruidModule(),
+//            new ExtendColumnDruidModule(),
+//            new DecimalDruidModule()
+    };
+
+    public static final ObjectMapper JSON_MAPPER;
+
+    static {
+        JSON_MAPPER = new DefaultObjectMapper();
+        for (DruidModule module : druidModules) {
+            for (Module jacksonModule : module.getJacksonModules()) {
+                JSON_MAPPER.registerModule(jacksonModule);
+            }
+        }
+        JSON_MAPPER.registerSubtypes(new NamedType(NumberedShardSpec.class, "numbered"));
+    }
+
+    //we will support distinctCount, ExtendColumn, Decimal metrics later
+//    public static void registerDruidSerde() {
+//        if (ComplexMetrics.getSerdeForType("kylin-distinctCount") == null) {
+//            ComplexMetrics.registerSerde("kylin-distinctCount", new DistinctCountSerde());
+//        }
+//
+//        if (ComplexMetrics.getSerdeForType(ExtendColumnAggregatorFactory.EXTEND_COLUMN) == null) {
+//            ComplexMetrics.registerSerde(ExtendColumnAggregatorFactory.EXTEND_COLUMN, new ExtendColumnSerde());
+//        }
+//
+//        if (ComplexMetrics.getSerdeForType(DecimalDruidModule.DECIMALSUM) == null) {
+//            ComplexMetrics.registerSerde(DecimalDruidModule.DECIMALSUM, new DecimalSumSerde());
+//        }
+//
+//        if (ComplexMetrics.getSerdeForType(DecimalDruidModule.DECIMALMIN) == null) {
+//            ComplexMetrics.registerSerde(DecimalDruidModule.DECIMALMIN, new DecimalMinSerde());
+//        }
+//
+//        if (ComplexMetrics.getSerdeForType(DecimalDruidModule.DECIMALMAX) == null) {
+//            ComplexMetrics.registerSerde(DecimalDruidModule.DECIMALMAX, new DecimalMaxSerde());
+//        }
+//    }
+
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidServerMetadata.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidServerMetadata.java
new file mode 100644
index 0000000000..66e598f82b
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/DruidServerMetadata.java
@@ -0,0 +1,85 @@
+/*
+ * 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.storage.druid.common;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class DruidServerMetadata {
+    private final String name;
+    private final String hostAndPort;
+    private final String hostAndTlsPort;
+    private final long maxSize;
+    private final String tier;
+    private final String type;
+    private final int priority;
+
+    @JsonCreator
+    public DruidServerMetadata(@JsonProperty("name") String name, @JsonProperty("host") String hostAndPort,
+            @JsonProperty("hostAndTlsPort") String hostAndTlsPort, @JsonProperty("maxSize") long maxSize,
+            @JsonProperty("type") String type, @JsonProperty("tier") String tier,
+            @JsonProperty("priority") int priority) {
+        this.name = name;
+        this.hostAndPort = hostAndPort;
+        this.hostAndTlsPort = hostAndTlsPort;
+        this.maxSize = maxSize;
+        this.tier = tier;
+        this.type = type;
+        this.priority = priority;
+    }
+
+    @JsonProperty
+    public String getName() {
+        return name;
+    }
+
+    public String getHost() {
+        return getHostAndTlsPort() != null ? getHostAndTlsPort() : getHostAndPort();
+    }
+
+    @JsonProperty("host")
+    public String getHostAndPort() {
+        return hostAndPort;
+    }
+
+    @JsonProperty
+    public String getHostAndTlsPort() {
+        return hostAndTlsPort;
+    }
+
+    @JsonProperty
+    public long getMaxSize() {
+        return maxSize;
+    }
+
+    @JsonProperty
+    public String getTier() {
+        return tier;
+    }
+
+    @JsonProperty
+    public String getType() {
+        return type;
+    }
+
+    @JsonProperty
+    public int getPriority() {
+        return priority;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ForeverDropRule.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ForeverDropRule.java
new file mode 100644
index 0000000000..268051f573
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ForeverDropRule.java
@@ -0,0 +1,29 @@
+/*
+ * 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.storage.druid.common;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class ForeverDropRule extends Rule {
+    @Override
+    @JsonProperty
+    String getType() {
+        return "dropForever";
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ForeverLoadRule.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ForeverLoadRule.java
new file mode 100644
index 0000000000..48fca64daf
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ForeverLoadRule.java
@@ -0,0 +1,61 @@
+/*
+ * 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.storage.druid.common;
+
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class ForeverLoadRule extends Rule {
+    private final Map<String, Integer> tieredReplicants;
+
+    @JsonCreator
+    public ForeverLoadRule(@JsonProperty("tieredReplicants") Map<String, Integer> tieredReplicants) {
+        this.tieredReplicants = tieredReplicants;
+        validateTieredReplicants(tieredReplicants);
+    }
+
+    @JsonProperty
+    @Override
+    public String getType() {
+        return "loadForever";
+    }
+
+    @JsonProperty
+    public Map<String, Integer> getTieredReplicants() {
+        return tieredReplicants;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ForeverLoadRule that = (ForeverLoadRule) o;
+        return Objects.equals(tieredReplicants, that.tieredReplicants);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(tieredReplicants);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ImmutableSegmentLoadInfo.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ImmutableSegmentLoadInfo.java
new file mode 100644
index 0000000000..6d4472f2c6
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/ImmutableSegmentLoadInfo.java
@@ -0,0 +1,82 @@
+/*
+ * 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.storage.druid.common;
+
+import java.util.Set;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+
+import io.druid.timeline.DataSegment;
+
+public class ImmutableSegmentLoadInfo {
+    private final DataSegment segment;
+    private final ImmutableSet<DruidServerMetadata> servers;
+
+    @JsonCreator
+    public ImmutableSegmentLoadInfo(@JsonProperty("segment") DataSegment segment,
+            @JsonProperty("servers") Set<DruidServerMetadata> servers) {
+        Preconditions.checkNotNull(segment, "segment");
+        Preconditions.checkNotNull(servers, "servers");
+        this.segment = segment;
+        this.servers = ImmutableSet.copyOf(servers);
+    }
+
+    @JsonProperty("segment")
+    public DataSegment getSegment() {
+        return segment;
+    }
+
+    @JsonProperty("servers")
+    public ImmutableSet<DruidServerMetadata> getServers() {
+        return servers;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        ImmutableSegmentLoadInfo that = (ImmutableSegmentLoadInfo) o;
+
+        if (!segment.equals(that.segment)) {
+            return false;
+        }
+        return servers.equals(that.servers);
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = segment.hashCode();
+        result = 31 * result + servers.hashCode();
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "SegmentLoadInfo{" + "segment=" + segment + ", servers=" + servers + '}';
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/MySQLConnector.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/MySQLConnector.java
new file mode 100644
index 0000000000..0bca01bb2f
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/MySQLConnector.java
@@ -0,0 +1,146 @@
+/*
+ * 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.storage.druid.common;
+
+import com.google.common.base.Predicate;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.mysql.jdbc.exceptions.MySQLTransientException;
+import io.druid.java.util.common.RetryUtils;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.kylin.common.KylinConfig;
+import org.skife.jdbi.v2.DBI;
+import org.skife.jdbi.v2.TransactionCallback;
+import org.skife.jdbi.v2.exceptions.DBIException;
+import org.skife.jdbi.v2.exceptions.UnableToExecuteStatementException;
+import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.SQLException;
+import java.sql.SQLRecoverableException;
+import java.sql.SQLTransientException;
+import java.util.concurrent.Callable;
+
+public class MySQLConnector {
+    protected static final Logger logger = LoggerFactory.getLogger(MySQLConnector.class);
+
+    private final DBI dbi;
+    private final Predicate<Throwable> shouldRetry;
+
+    public MySQLConnector() {
+        final BasicDataSource datasource = getDatasource();
+        // MySQL driver is classloader isolated as part of the extension
+        // so we need to help JDBC find the driver
+        datasource.setDriverClassLoader(getClass().getClassLoader());
+        datasource.setDriverClassName("com.mysql.jdbc.Driver");
+
+        // use double-quotes for quoting columns, so we can write SQL that works with most databases
+        datasource.setConnectionInitSqls(ImmutableList.of("SET sql_mode='ANSI_QUOTES'"));
+
+        this.dbi = new DBI(datasource);
+
+        this.shouldRetry = new Predicate<Throwable>() {
+            @Override
+            public boolean apply(Throwable e) {
+                return isTransientException(e);
+            }
+        };
+
+        logger.info("Configured MySQL as metadata storage");
+    }
+
+    public <T> T retryTransaction(final TransactionCallback<T> callback, final int quietTries, final int maxTries) {
+        final Callable<T> call = new Callable<T>() {
+            @Override
+            public T call() throws Exception {
+                return getDBI().inTransaction(callback);
+            }
+        };
+        try {
+            return RetryUtils.retry(call, shouldRetry, quietTries, maxTries);
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+
+    public final boolean isTransientException(Throwable e) {
+        if (e == null) {
+            return false;
+        }
+
+        if (e instanceof RetryTransactionException) {
+            return true;
+        }
+
+        if (e instanceof SQLTransientException) {
+            return true;
+        }
+
+        if (e instanceof SQLRecoverableException) {
+            return true;
+        }
+        if (e instanceof UnableToObtainConnectionException) {
+            return true;
+        }
+
+        if (e instanceof UnableToExecuteStatementException) {
+            return true;
+        }
+
+        if (connectorIsTransientException(e)) {
+            return true;
+        }
+
+        if (e instanceof SQLException && isTransientException(e.getCause())) {
+            return true;
+        }
+
+        if (e instanceof DBIException && isTransientException(e.getCause())) {
+            return true;
+        }
+
+        return false;
+    }
+
+    protected boolean connectorIsTransientException(Throwable e) {
+        return e instanceof MySQLTransientException || (e instanceof SQLException && ((SQLException) e).getErrorCode() == 1317 /* ER_QUERY_INTERRUPTED */);
+    }
+
+    public DBI getDBI() {
+        return dbi;
+    }
+
+    private BasicDataSource getDatasource() {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        BasicDataSource dataSource = new BasicDataSource();
+        dataSource.setUsername(config.getDruidMysqlUser());
+        dataSource.setPassword(config.getDruidMysqlPassword());
+        dataSource.setUrl(config.getDruidMysqlUrl());
+
+        dataSource.setValidationQuery(getValidationQuery());
+        dataSource.setTestOnBorrow(true);
+
+        return dataSource;
+    }
+
+    private String getValidationQuery() {
+        return "SELECT 1";
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/NumberedShardSpec.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/NumberedShardSpec.java
new file mode 100644
index 0000000000..fae5e07786
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/NumberedShardSpec.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.kylin.storage.druid.common;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import io.druid.data.input.InputRow;
+import io.druid.timeline.partition.NumberedPartitionChunk;
+import io.druid.timeline.partition.PartitionChunk;
+import io.druid.timeline.partition.ShardSpec;
+import io.druid.timeline.partition.ShardSpecLookup;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * copy from druid
+ */
+@JsonTypeName("numbered")
+public class NumberedShardSpec implements ShardSpec {
+    @JsonIgnore
+    final private int partitionNum;
+
+    @JsonIgnore
+    final private int partitions;
+
+    @JsonCreator
+    public NumberedShardSpec(@JsonProperty("partitionNum") int partitionNum, @JsonProperty("partitions") int partitions) {
+        Preconditions.checkArgument(partitionNum >= 0, "partitionNum >= 0");
+        Preconditions.checkArgument(partitions >= 0, "partitions >= 0");
+        this.partitionNum = partitionNum;
+        this.partitions = partitions;
+    }
+
+    @JsonProperty("partitionNum")
+    @Override
+    public int getPartitionNum() {
+        return partitionNum;
+    }
+
+    @Override
+    public ShardSpecLookup getLookup(final List<ShardSpec> shardSpecs) {
+        return new ShardSpecLookup() {
+            @Override
+            public ShardSpec getShardSpec(long timestamp, InputRow row) {
+                return shardSpecs.get(0);
+            }
+        };
+    }
+
+    @Override
+    public Map<String, Range<String>> getDomain() {
+        return ImmutableMap.of();
+    }
+
+    @JsonProperty("partitions")
+    public int getPartitions() {
+        return partitions;
+    }
+
+    @Override
+    public <T> PartitionChunk<T> createChunk(T obj) {
+        return NumberedPartitionChunk.make(partitionNum, partitions, obj);
+    }
+
+    @Override
+    public boolean isInChunk(long timestamp, InputRow inputRow) {
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "NumberedShardSpec{" + "partitionNum=" + partitionNum + ", partitions=" + partitions + '}';
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/PeriodLoadRule.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/PeriodLoadRule.java
new file mode 100644
index 0000000000..1e765b3a22
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/PeriodLoadRule.java
@@ -0,0 +1,70 @@
+/*
+ * 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.storage.druid.common;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.joda.time.Period;
+
+import java.util.Map;
+import java.util.Objects;
+
+public class PeriodLoadRule extends Rule {
+    private final Period period;
+    private final Map<String, Integer> tieredReplicants;
+
+    @JsonCreator
+    public PeriodLoadRule(
+            @JsonProperty("period") Period period,
+            @JsonProperty("tieredReplicants") Map<String, Integer> tieredReplicants) {
+        this.period = period;
+        this.tieredReplicants = tieredReplicants;
+        validateTieredReplicants(tieredReplicants);
+    }
+
+    @Override
+    @JsonProperty
+    public String getType() {
+        return "loadByPeriod";
+    }
+
+    @JsonProperty
+    public Period getPeriod() {
+        return period;
+    }
+
+    @JsonProperty
+    public Map<String, Integer> getTieredReplicants() {
+        return tieredReplicants;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof PeriodLoadRule)) return false;
+        PeriodLoadRule that = (PeriodLoadRule) o;
+        return Objects.equals(period, that.period) &&
+                Objects.equals(tieredReplicants, that.tieredReplicants);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(period, tieredReplicants);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/RetryTransactionException.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/RetryTransactionException.java
new file mode 100644
index 0000000000..c769d2494f
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/RetryTransactionException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.storage.druid.common;
+
+/**
+ * Exception thrown by SQL connector code when it wants a transaction to be retried. This exception is checked for
+ * by SQLMetadataConnector#isTransientException(Throwable)}.
+ */
+public class RetryTransactionException extends RuntimeException {
+    public RetryTransactionException(String message) {
+        super(message);
+    }
+}
\ No newline at end of file
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/Rule.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/Rule.java
new file mode 100644
index 0000000000..a6dad31a54
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/Rule.java
@@ -0,0 +1,49 @@
+/*
+ * 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.storage.druid.common;
+
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.druid.java.util.common.IAE;
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+@JsonSubTypes(value = {
+        @JsonSubTypes.Type(name = "loadByPeriod", value = PeriodLoadRule.class),
+        @JsonSubTypes.Type(name = "loadForever", value = ForeverLoadRule.class),
+        @JsonSubTypes.Type(name = "dropForever", value = ForeverDropRule.class)
+})
+public abstract class Rule {
+    abstract String getType();
+
+    protected void validateTieredReplicants(Map<String, Integer> tieredReplicants) {
+        if (tieredReplicants.size() == 0) {
+            throw new IAE("A rule with empty tiered replicants is invalid");
+        }
+        for (Map.Entry<String, Integer> entry : tieredReplicants.entrySet()) {
+            if (entry.getValue() == null) {
+                throw new IAE("Replicant value cannot be empty");
+            }
+            if (entry.getValue() < 0) {
+                throw new IAE("Replicant value [%d] is less than 0, which is not allowed", entry.getValue());
+            }
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/SegmentPublishResult.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/SegmentPublishResult.java
new file mode 100644
index 0000000000..5d11efa79e
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/common/SegmentPublishResult.java
@@ -0,0 +1,75 @@
+/*
+ * 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.storage.druid.common;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import io.druid.timeline.DataSegment;
+
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * copy from druid
+ */
+public class SegmentPublishResult {
+    private final Set<DataSegment> segments;
+    private final boolean success;
+
+    public SegmentPublishResult(Set<DataSegment> segments, boolean success) {
+        this.segments = Preconditions.checkNotNull(segments, "segments");
+        this.success = success;
+
+        if (!success) {
+            Preconditions.checkArgument(segments.isEmpty(), "segments must be empty for unsuccessful publishes");
+        }
+    }
+
+    @JsonProperty
+    public Set<DataSegment> getSegments() {
+        return segments;
+    }
+
+    @JsonProperty
+    public boolean isSuccess() {
+        return success;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        SegmentPublishResult that = (SegmentPublishResult) o;
+        return success == that.success && Objects.equals(segments, that.segments);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(segments, success);
+    }
+
+    @Override
+    public String toString() {
+        return "SegmentPublishResult{" + "segments=" + segments + ", success=" + success + '}';
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/AbstractHttpClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/AbstractHttpClient.java
new file mode 100644
index 0000000000..33367b0f41
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/AbstractHttpClient.java
@@ -0,0 +1,31 @@
+/*
+ * 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.storage.druid.http;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public abstract class AbstractHttpClient implements HttpClient {
+    @Override
+    public <Intermediate, Final> ListenableFuture<Final> go(
+            final Request request,
+            final HttpResponseHandler<Intermediate, Final> handler
+    ) {
+        return go(request, handler, null);
+    }
+}
\ No newline at end of file
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/ClientResponse.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/ClientResponse.java
new file mode 100644
index 0000000000..af55e15f51
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/ClientResponse.java
@@ -0,0 +1,48 @@
+/*
+ * 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.storage.druid.http;
+
+public class ClientResponse<T> {
+    private final boolean finished;
+    private final T obj;
+
+    protected ClientResponse(
+            boolean finished,
+            T obj
+    ) {
+        this.finished = finished;
+        this.obj = obj;
+    }
+
+    public static <T> ClientResponse<T> finished(T obj) {
+        return new ClientResponse<T>(true, obj);
+    }
+
+    public static <T> ClientResponse<T> unfinished(T obj) {
+        return new ClientResponse<T>(false, obj);
+    }
+
+    public boolean isFinished() {
+        return finished;
+    }
+
+    public T getObj() {
+        return obj;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/FullResponseHandler.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/FullResponseHandler.java
new file mode 100644
index 0000000000..5cbcca4a3a
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/FullResponseHandler.java
@@ -0,0 +1,60 @@
+/*
+ * 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.storage.druid.http;
+
+import java.nio.charset.Charset;
+
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+public class FullResponseHandler implements HttpResponseHandler<FullResponseHolder, FullResponseHolder> {
+    private final Charset charset;
+
+    public FullResponseHandler(Charset charset) {
+        this.charset = charset;
+    }
+
+    @Override
+    public ClientResponse<FullResponseHolder> handleResponse(HttpResponse response) {
+        return ClientResponse.unfinished(new FullResponseHolder(response.getStatus(), response, new StringBuilder(response.getContent().toString(charset))));
+    }
+
+    @Override
+    public ClientResponse<FullResponseHolder> handleChunk(ClientResponse<FullResponseHolder> response, HttpChunk chunk) {
+        final StringBuilder builder = response.getObj().getBuilder();
+
+        if (builder == null) {
+            return ClientResponse.finished(null);
+        }
+
+        builder.append(chunk.getContent().toString(charset));
+        return response;
+    }
+
+    @Override
+    public ClientResponse<FullResponseHolder> done(ClientResponse<FullResponseHolder> response) {
+        return ClientResponse.finished(response.getObj());
+    }
+
+    @Override
+    public void exceptionCaught(ClientResponse<FullResponseHolder> clientResponse, Throwable e) {
+        // Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
+    }
+
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/FullResponseHolder.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/FullResponseHolder.java
new file mode 100644
index 0000000000..9aaa2b6b30
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/FullResponseHolder.java
@@ -0,0 +1,50 @@
+/*
+ * 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.storage.druid.http;
+
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+
+public class FullResponseHolder {
+    private final HttpResponseStatus status;
+    private final HttpResponse response;
+    private final StringBuilder builder;
+
+    public FullResponseHolder(HttpResponseStatus status, HttpResponse response, StringBuilder builder) {
+        this.status = status;
+        this.response = response;
+        this.builder = builder;
+    }
+
+    public HttpResponseStatus getStatus() {
+        return status;
+    }
+
+    public HttpResponse getResponse() {
+        return response;
+    }
+
+    public StringBuilder getBuilder() {
+        return builder;
+    }
+
+    public String getContent() {
+        return builder.toString();
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClient.java
new file mode 100644
index 0000000000..1e7fc0f907
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClient.java
@@ -0,0 +1,66 @@
+/*
+ * 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.storage.druid.http;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.joda.time.Duration;
+
+/**
+ * Interface for Async HTTP client libraries.
+ */
+public interface HttpClient {
+    /**
+     * Submit a request and process the response with the given response handler.
+     * <p>
+     * Note that the Request object passed in to the HttpClient *may* be mutated by the actual client.  This is
+     * largely done by composed clients, but the contract is that mutation is possible.  It is the caller's
+     * responsibility to pass in a copy of the Request object if they want to have an object that is not mutated.
+     *
+     * @param request        Request to process, this *may* be mutated by the client
+     * @param handler        An asynchronous response handler that will be used to process results from the http call
+     * @param <Intermediate> The type of the intermediate results from the handler
+     * @param <Final>        The type of the final results that the returned ListenableFuture will contain
+     * @return A listenable future that will eventually provide an object of type Final
+     */
+    public <Intermediate, Final> ListenableFuture<Final> go(
+            Request request,
+            HttpResponseHandler<Intermediate, Final> handler
+    );
+
+    /**
+     * Submit a request and process the response with the given response handler.
+     * <p>
+     * Note that the Request object passed in to the HttpClient *may* be mutated by the actual client.  This is
+     * largely done by composed clients, but the contract is that mutation is possible.  It is the caller's
+     * responsibility to pass in a copy of the Request object if they want to have an object that is not mutated.
+     *
+     * @param request        Request to process, this *may* be mutated by the client
+     * @param handler        An asynchronous response handler that will be used to process results from the http call
+     * @param readTimeout    Read timeout to use for this request. Leave null to use the default readTimeout. Set to zero
+     *                       to disable timeouts for this request.
+     * @param <Intermediate> The type of the intermediate results from the handler
+     * @param <Final>        The type of the final results that the returned ListenableFuture will contain
+     * @return A listenable future that will eventually provide an object of type Final
+     */
+    public <Intermediate, Final> ListenableFuture<Final> go(
+            Request request,
+            HttpResponseHandler<Intermediate, Final> handler,
+            Duration readTimeout
+    );
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientConfig.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientConfig.java
new file mode 100644
index 0000000000..6f6d4761a0
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientConfig.java
@@ -0,0 +1,149 @@
+/*
+ * 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.storage.druid.http;
+
+import org.joda.time.Duration;
+
+public class HttpClientConfig {
+    public static final CompressionCodec DEFAULT_COMPRESSION_CODEC = CompressionCodec.GZIP;
+    // Default from NioClientSocketChannelFactory.DEFAULT_BOSS_COUNT, which is private:
+    private static final int DEFAULT_BOSS_COUNT = 1;
+    // Default from SelectorUtil.DEFAULT_IO_THREADS, which is private:
+    private static final int DEFAULT_WORKER_COUNT = Runtime.getRuntime().availableProcessors() * 2;
+    private final int numConnections;
+    private final Duration readTimeout;
+    private final int bossPoolSize;
+    private final int workerPoolSize;
+    private final CompressionCodec compressionCodec;
+
+    private HttpClientConfig(
+            int numConnections,
+            Duration readTimeout,
+            int bossPoolSize,
+            int workerPoolSize,
+            CompressionCodec compressionCodec
+    ) {
+        this.numConnections = numConnections;
+        this.readTimeout = readTimeout;
+        this.bossPoolSize = bossPoolSize;
+        this.workerPoolSize = workerPoolSize;
+        this.compressionCodec = compressionCodec;
+    }
+
+    public static Builder builder() {
+        return new Builder();
+    }
+
+    public int getNumConnections() {
+        return numConnections;
+    }
+
+    public Duration getReadTimeout() {
+        return readTimeout;
+    }
+
+    public int getBossPoolSize() {
+        return bossPoolSize;
+    }
+
+    public int getWorkerPoolSize() {
+        return workerPoolSize;
+    }
+
+    public CompressionCodec getCompressionCodec() {
+        return compressionCodec;
+    }
+
+    public enum CompressionCodec {
+        IDENTITY {
+            @Override
+            public String getEncodingString() {
+                return "identity";
+            }
+        },
+        GZIP {
+            @Override
+            public String getEncodingString() {
+                return "gzip";
+            }
+        },
+        DEFLATE {
+            @Override
+            public String getEncodingString() {
+                return "deflate";
+            }
+        };
+
+        /**
+         * Get the header-ified name of this encoding, which should go in "Accept-Encoding" and
+         * "Content-Encoding" headers. This is not just the lowercasing of the enum name, since
+         * we may one day support x- encodings like LZ4, which would likely be an enum named
+         * "LZ4" that has an encoding string like "x-lz4".
+         *
+         * @return encoding name
+         */
+        public abstract String getEncodingString();
+    }
+
+    public static class Builder {
+        private int numConnections = 1;
+        private Duration readTimeout = null;
+        private int bossCount = DEFAULT_BOSS_COUNT;
+        private int workerCount = DEFAULT_WORKER_COUNT;
+        private CompressionCodec compressionCodec = DEFAULT_COMPRESSION_CODEC;
+
+        private Builder() {
+        }
+
+        public Builder withNumConnections(int numConnections) {
+            this.numConnections = numConnections;
+            return this;
+        }
+
+        public Builder withReadTimeout(Duration readTimeout) {
+            this.readTimeout = readTimeout;
+            return this;
+        }
+
+        public Builder withBossCount(int bossCount) {
+            this.bossCount = bossCount;
+            return this;
+        }
+
+        public Builder withWorkerCount(int workerCount) {
+            this.workerCount = workerCount;
+            return this;
+        }
+
+        public Builder withCompressionCodec(CompressionCodec compressionCodec) {
+            this.compressionCodec = compressionCodec;
+            return this;
+        }
+
+        public HttpClientConfig build() {
+            return new HttpClientConfig(
+                    numConnections,
+                    readTimeout,
+                    bossCount,
+                    workerCount,
+                    compressionCodec
+            );
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientFactory.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientFactory.java
new file mode 100644
index 0000000000..dd5ba7e3d5
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientFactory.java
@@ -0,0 +1,93 @@
+/*
+ * 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.storage.druid.http;
+
+import com.google.common.base.Throwables;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.kylin.storage.druid.http.pool.ChannelResourceFactory;
+import org.apache.kylin.storage.druid.http.pool.ResourcePool;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.socket.nio.NioClientBossPool;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioWorkerPool;
+import org.jboss.netty.logging.InternalLoggerFactory;
+import org.jboss.netty.logging.Slf4JLoggerFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.ThreadNameDeterminer;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+public class HttpClientFactory {
+
+    public static HttpClient createClient(HttpClientConfig config) {
+        try {
+            final HashedWheelTimer timer = new HashedWheelTimer(
+                    new ThreadFactoryBuilder().setDaemon(true)
+                            .setNameFormat("HttpClient-Timer-%s")
+                            .build(),
+                    100,
+                    TimeUnit.MILLISECONDS
+            );
+            timer.start();
+
+            final NioClientBossPool bossPool = new NioClientBossPool(
+                    Executors.newCachedThreadPool(
+                            new ThreadFactoryBuilder()
+                                    .setDaemon(true)
+                                    .setNameFormat("HttpClient-Netty-Boss-%s")
+                                    .build()
+                    ),
+                    config.getBossPoolSize(),
+                    timer,
+                    ThreadNameDeterminer.CURRENT
+            );
+
+            final NioWorkerPool workerPool = new NioWorkerPool(
+                    Executors.newCachedThreadPool(
+                            new ThreadFactoryBuilder()
+                                    .setDaemon(true)
+                                    .setNameFormat("HttpClient-Netty-Worker-%s")
+                                    .build()
+                    ),
+                    config.getWorkerPoolSize(),
+                    ThreadNameDeterminer.CURRENT
+            );
+
+            final ClientBootstrap bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(bossPool, workerPool));
+            bootstrap.setOption("keepAlive", true);
+            bootstrap.setPipelineFactory(new HttpClientPipelineFactory());
+
+            InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+
+            return new NettyHttpClient(
+                    new ResourcePool<>(
+                            new ChannelResourceFactory(bootstrap),
+                            config.getNumConnections()
+                    ),
+                    config.getReadTimeout(),
+                    config.getCompressionCodec(),
+                    timer
+            );
+
+        } catch (Exception e) {
+            throw Throwables.propagate(e);
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientPipelineFactory.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientPipelineFactory.java
new file mode 100644
index 0000000000..e8dbe84e85
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpClientPipelineFactory.java
@@ -0,0 +1,37 @@
+/*
+ * 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.storage.druid.http;
+
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.DefaultChannelPipeline;
+import org.jboss.netty.handler.codec.http.HttpClientCodec;
+import org.jboss.netty.handler.codec.http.HttpContentDecompressor;
+
+public class HttpClientPipelineFactory implements ChannelPipelineFactory {
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+        ChannelPipeline pipeline = new DefaultChannelPipeline();
+
+        pipeline.addLast("codec", new HttpClientCodec());
+        pipeline.addLast("inflater", new HttpContentDecompressor());
+
+        return pipeline;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpResponseHandler.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpResponseHandler.java
new file mode 100644
index 0000000000..136b1ba462
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/HttpResponseHandler.java
@@ -0,0 +1,55 @@
+/*
+ * 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.storage.druid.http;
+
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+
+/**
+ * A handler for an HTTP request.
+ * <p>
+ * The ClientResponse object passed around is used to store state between further chunks and indicate when it is safe
+ * to hand the object back to the caller.
+ * <p>
+ * If the response is chunked, the ClientResponse object returned from handleResponse will be passed in as the
+ * first argument to handleChunk().
+ * <p>
+ * If the ClientResponse object is marked as finished, that indicates that the object stored is safe to hand
+ * off to the caller.  This is most often done either from the done() method after all content has been processed or
+ * from the initial handleResponse method to indicate that the object is thread-safe and aware that it might be
+ * accessed before all chunks come back.
+ * <p>
+ * Note: if you return a finished ClientResponse object from anything other than the done() method, IntermediateType
+ * must be castable to FinalType
+ */
+public interface HttpResponseHandler<IntermediateType, FinalType> {
+    /**
+     * Handles the initial HttpResponse object that comes back from Netty.
+     *
+     * @param response - response from Netty
+     * @return
+     */
+    public ClientResponse<IntermediateType> handleResponse(HttpResponse response);
+
+    public ClientResponse<IntermediateType> handleChunk(ClientResponse<IntermediateType> clientResponse, HttpChunk chunk);
+
+    public ClientResponse<FinalType> done(ClientResponse<IntermediateType> clientResponse);
+
+    public void exceptionCaught(ClientResponse<IntermediateType> clientResponse, Throwable e);
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/NettyHttpClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/NettyHttpClient.java
new file mode 100644
index 0000000000..7f22028143
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/NettyHttpClient.java
@@ -0,0 +1,357 @@
+/*
+ * 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.storage.druid.http;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Multimap;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.kylin.storage.druid.http.pool.ResourceContainer;
+import org.apache.kylin.storage.druid.http.pool.ResourcePool;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.handler.codec.http.DefaultHttpRequest;
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpVersion;
+import org.jboss.netty.handler.timeout.ReadTimeoutHandler;
+import org.jboss.netty.util.Timer;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.util.Collection;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class NettyHttpClient extends AbstractHttpClient {
+    private static final Logger log = LoggerFactory.getLogger(NettyHttpClient.class);
+
+    private static final String READ_TIMEOUT_HANDLER_NAME = "read-timeout";
+    private static final String LAST_HANDLER_NAME = "last-handler";
+
+    private final Timer timer;
+    private final ResourcePool<String, ChannelFuture> pool;
+    private final HttpClientConfig.CompressionCodec compressionCodec;
+    private final Duration defaultReadTimeout;
+
+    public NettyHttpClient(ResourcePool<String, ChannelFuture> pool) {
+        this(pool, null, HttpClientConfig.DEFAULT_COMPRESSION_CODEC, null);
+    }
+
+    NettyHttpClient(
+            ResourcePool<String, ChannelFuture> pool,
+            Duration defaultReadTimeout,
+            HttpClientConfig.CompressionCodec compressionCodec,
+            Timer timer
+    ) {
+        this.pool = Preconditions.checkNotNull(pool, "pool");
+        this.defaultReadTimeout = defaultReadTimeout;
+        this.compressionCodec = Preconditions.checkNotNull(compressionCodec);
+        this.timer = timer;
+
+        if (defaultReadTimeout != null && defaultReadTimeout.getMillis() > 0) {
+            Preconditions.checkNotNull(timer, "timer");
+        }
+    }
+
+    public void start() {
+    }
+
+    public void stop() {
+        pool.close();
+    }
+
+
+    @SuppressWarnings("checkstyle:methodlength")
+    @Override
+    public <Intermediate, Final> ListenableFuture<Final> go(
+            final Request request,
+            final HttpResponseHandler<Intermediate, Final> handler,
+            final Duration requestReadTimeout
+    ) {
+        final HttpMethod method = request.getMethod();
+        final URL url = request.getUrl();
+        final Multimap<String, String> headers = request.getHeaders();
+
+        final String requestDesc = String.format(Locale.ROOT, "%s %s", method, url);
+        if (log.isDebugEnabled()) {
+            log.debug("[{}] starting", requestDesc);
+        }
+
+        // Block while acquiring a channel from the pool, then complete the request asynchronously.
+        final Channel channel;
+        final String hostKey = getPoolKey(url);
+        final ResourceContainer<ChannelFuture> channelResourceContainer = pool.take(hostKey);
+        final ChannelFuture channelFuture = channelResourceContainer.get().awaitUninterruptibly();
+        if (!channelFuture.isSuccess()) {
+            channelResourceContainer.returnResource(); // Some other poor sap will have to deal with it...
+            return Futures.immediateFailedFuture(
+                    new ChannelException(
+                            "Faulty channel in resource pool",
+                            channelFuture.getCause()
+                    )
+            );
+        } else {
+            channel = channelFuture.getChannel();
+        }
+
+        final String urlFile = Strings.nullToEmpty(url.getFile());
+        final HttpRequest httpRequest = new DefaultHttpRequest(
+                HttpVersion.HTTP_1_1,
+                method,
+                urlFile.isEmpty() ? "/" : urlFile
+        );
+
+        if (!headers.containsKey(HttpHeaders.Names.HOST)) {
+            httpRequest.addHeader(HttpHeaders.Names.HOST, getHost(url));
+        }
+
+        // If Accept-Encoding is set in the Request, use that. Otherwise use the default from "compressionCodec".
+        if (!headers.containsKey(HttpHeaders.Names.ACCEPT_ENCODING)) {
+            httpRequest.setHeader(HttpHeaders.Names.ACCEPT_ENCODING, compressionCodec.getEncodingString());
+        }
+
+        for (Map.Entry<String, Collection<String>> entry : headers.asMap().entrySet()) {
+            String key = entry.getKey();
+
+            for (String obj : entry.getValue()) {
+                httpRequest.addHeader(key, obj);
+            }
+        }
+
+        if (request.hasContent()) {
+            httpRequest.setContent(request.getContent());
+        }
+
+        final long readTimeout = getReadTimeout(requestReadTimeout);
+        final SettableFuture<Final> retVal = SettableFuture.create();
+
+        if (readTimeout > 0) {
+            channel.getPipeline().addLast(
+                    READ_TIMEOUT_HANDLER_NAME,
+                    new ReadTimeoutHandler(timer, readTimeout, TimeUnit.MILLISECONDS)
+            );
+        }
+
+        channel.getPipeline().addLast(
+                LAST_HANDLER_NAME,
+                new SimpleChannelUpstreamHandler() {
+                    private volatile ClientResponse<Intermediate> response = null;
+
+                    @Override
+                    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
+                        if (log.isDebugEnabled()) {
+                            log.debug("[{}] messageReceived: {}", requestDesc, e.getMessage());
+                        }
+                        try {
+                            Object msg = e.getMessage();
+
+                            if (msg instanceof HttpResponse) {
+                                HttpResponse httpResponse = (HttpResponse) msg;
+                                if (log.isDebugEnabled()) {
+                                    log.debug("[{}] Got response: {}", requestDesc, httpResponse.getStatus());
+                                }
+
+                                response = handler.handleResponse(httpResponse);
+                                if (response.isFinished()) {
+                                    retVal.set((Final) response.getObj());
+                                }
+
+                                if (!httpResponse.isChunked()) {
+                                    finishRequest();
+                                }
+                            } else if (msg instanceof HttpChunk) {
+                                HttpChunk httpChunk = (HttpChunk) msg;
+                                if (log.isDebugEnabled()) {
+                                    log.debug(
+                                            "[{}] Got chunk: {}B, last={}",
+                                            requestDesc,
+                                            httpChunk.getContent().readableBytes(),
+                                            httpChunk.isLast()
+                                    );
+                                }
+
+                                if (httpChunk.isLast()) {
+                                    finishRequest();
+                                } else {
+                                    response = handler.handleChunk(response, httpChunk);
+                                    if (response.isFinished() && !retVal.isDone()) {
+                                        retVal.set((Final) response.getObj());
+                                    }
+                                }
+                            } else {
+                                throw new IllegalStateException(String.format(Locale.ROOT, "Unknown message type[%s]", msg.getClass()));
+                            }
+                        } catch (Exception ex) {
+                            log.warn(String.format(Locale.ROOT, "[%s] Exception thrown while processing message, closing channel.", requestDesc), ex);
+
+                            // FIXME no need to set because exceptionCaught calls retVal.setException?
+                            if (!retVal.isDone()) {
+                                retVal.set(null);
+                            }
+                            channel.close();
+                            channelResourceContainer.returnResource();
+
+                            throw ex;
+                        }
+                    }
+
+                    private void finishRequest() {
+                        ClientResponse<Final> finalResponse = handler.done(response);
+                        if (!finalResponse.isFinished()) {
+                            throw new IllegalStateException(
+                                    String.format(Locale.ROOT,
+                                            "[%s] Didn't get a completed ClientResponse Object from [%s]",
+                                            requestDesc,
+                                            handler.getClass()
+                                    )
+                            );
+                        }
+                        if (!retVal.isDone()) {
+                            retVal.set(finalResponse.getObj());
+                        }
+                        removeHandlers();
+                        channelResourceContainer.returnResource();
+                    }
+
+                    @Override
+                    public void exceptionCaught(ChannelHandlerContext context, ExceptionEvent event) throws Exception {
+                        if (log.isDebugEnabled()) {
+                            final Throwable cause = event.getCause();
+                            if (cause == null) {
+                                log.debug(String.format(Locale.ROOT, "[%s] Caught exception", requestDesc));
+                            } else {
+                                log.debug(String.format(Locale.ROOT, "[%s] Caught exception", requestDesc), cause);
+                            }
+                        }
+
+                        retVal.setException(event.getCause());
+                        // response is non-null if we received initial chunk and then exception occurs
+                        if (response != null) {
+                            handler.exceptionCaught(response, event.getCause());
+                        }
+                        removeHandlers();
+                        try {
+                            channel.close();
+                        } catch (Exception e) {
+                            // ignore
+                        } finally {
+                            channelResourceContainer.returnResource();
+                        }
+
+                        context.sendUpstream(event);
+                    }
+
+                    @Override
+                    public void channelDisconnected(ChannelHandlerContext context, ChannelStateEvent event) throws Exception {
+                        if (log.isDebugEnabled()) {
+                            log.debug("[{}] Channel disconnected", requestDesc);
+                        }
+                        // response is non-null if we received initial chunk and then exception occurs
+                        if (response != null) {
+                            handler.exceptionCaught(response, new ChannelException("Channel disconnected"));
+                        }
+                        channel.close();
+                        channelResourceContainer.returnResource();
+                        if (!retVal.isDone()) {
+                            log.warn("[{}] Channel disconnected before response complete", requestDesc);
+                            retVal.setException(new ChannelException("Channel disconnected"));
+                        }
+                        context.sendUpstream(event);
+                    }
+
+                    private void removeHandlers() {
+                        if (readTimeout > 0) {
+                            channel.getPipeline().remove(READ_TIMEOUT_HANDLER_NAME);
+                        }
+                        channel.getPipeline().remove(LAST_HANDLER_NAME);
+                    }
+                }
+        );
+
+        channel.write(httpRequest).addListener(
+                new ChannelFutureListener() {
+                    @Override
+                    public void operationComplete(ChannelFuture future) throws Exception {
+                        if (!future.isSuccess()) {
+                            channel.close();
+                            channelResourceContainer.returnResource();
+                            if (!retVal.isDone()) {
+                                retVal.setException(
+                                        new ChannelException(
+                                                String.format(Locale.ROOT, "[%s] Failed to write request to channel", requestDesc),
+                                                future.getCause()
+                                        )
+                                );
+                            }
+                        }
+                    }
+                }
+        );
+
+        return retVal;
+    }
+
+    private long getReadTimeout(Duration requestReadTimeout) {
+        final long timeout;
+        if (requestReadTimeout != null) {
+            timeout = requestReadTimeout.getMillis();
+        } else if (defaultReadTimeout != null) {
+            timeout = defaultReadTimeout.getMillis();
+        } else {
+            timeout = 0;
+        }
+
+        if (timeout > 0 && timer == null) {
+            log.warn("Cannot time out requests without a timer! Disabling timeout for this request.");
+            return 0;
+        } else {
+            return timeout;
+        }
+    }
+
+    private String getHost(URL url) {
+        if (!"http".equalsIgnoreCase(url.getProtocol())) {
+            throw new IllegalArgumentException("Invalid url protocol:" + url.getProtocol());
+        }
+        int port = url.getPort() == -1 ? url.getDefaultPort() : url.getPort();
+        return String.format(Locale.ROOT, "%s:%s", url.getHost(), port);
+    }
+
+    private String getPoolKey(URL url) {
+        return String.format(Locale.ROOT,
+                "%s://%s:%s", url.getProtocol(), url.getHost(), url.getPort() == -1 ? url.getDefaultPort() : url.getPort()
+        );
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/Request.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/Request.java
new file mode 100644
index 0000000000..edfa2bf10a
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/Request.java
@@ -0,0 +1,170 @@
+/*
+ * 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.storage.druid.http;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferFactory;
+import org.jboss.netty.buffer.HeapChannelBufferFactory;
+import org.jboss.netty.handler.codec.base64.Base64;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class Request {
+    private static final ChannelBufferFactory factory = HeapChannelBufferFactory.getInstance();
+
+    private final HttpMethod method;
+    private final URL url;
+    private final Multimap<String, String> headers = Multimaps.newListMultimap(
+            Maps.<String, Collection<String>>newHashMap(),
+            new Supplier<List<String>>() {
+                @Override
+                public List<String> get() {
+                    return Lists.newArrayList();
+                }
+            }
+    );
+
+    private ChannelBuffer content;
+
+    public Request(HttpMethod method, URL url) {
+        this.method = method;
+        this.url = url;
+    }
+
+    public HttpMethod getMethod() {
+        return method;
+    }
+
+    public URL getUrl() {
+        return url;
+    }
+
+    public Multimap<String, String> getHeaders() {
+        return headers;
+    }
+
+    public boolean hasContent() {
+        return content != null;
+    }
+
+    public ChannelBuffer getContent() {
+        return content;
+    }
+
+    public Request setContent(byte[] bytes) {
+        return setContent(null, bytes);
+    }
+
+    public Request setContent(ChannelBuffer content) {
+        return setContent(null, content);
+    }
+
+    public Request copy() {
+        Request retVal = new Request(method, url);
+        retVal.headers.putAll(this.headers);
+        retVal.content = content == null ? null : content.copy();
+        return retVal;
+    }
+
+    public Request setHeader(String header, String value) {
+        headers.replaceValues(header, Arrays.asList(value));
+        return this;
+    }
+
+    public Request setHeaderValues(String header, Iterable<String> value) {
+        headers.replaceValues(header, value);
+        return this;
+    }
+
+    public Request setHeaderValues(Multimap<String, String> inHeaders) {
+        for (Map.Entry<String, Collection<String>> entry : inHeaders.asMap().entrySet()) {
+            this.setHeaderValues(entry.getKey(), entry.getValue());
+        }
+        return this;
+    }
+
+    public Request addHeader(String header, String value) {
+        headers.put(header, value);
+        return this;
+    }
+
+    public Request addHeaderValues(String header, Iterable<String> value) {
+        headers.putAll(header, value);
+        return this;
+    }
+
+    public Request addHeaderValues(Multimap<String, String> inHeaders) {
+        for (Map.Entry<String, Collection<String>> entry : inHeaders.asMap().entrySet()) {
+            this.addHeaderValues(entry.getKey(), entry.getValue());
+        }
+        return this;
+    }
+
+    public Request setContent(byte[] bytes, int offset, int length) {
+        return setContent(null, bytes, offset, length);
+    }
+
+    public Request setContent(String contentType, byte[] bytes) {
+        return setContent(contentType, bytes, 0, bytes.length);
+    }
+
+    public Request setContent(String contentType, byte[] bytes, int offset, int length) {
+        return setContent(contentType, factory.getBuffer(bytes, offset, length));
+    }
+
+    public Request setContent(String contentType, ChannelBuffer content) {
+        if (contentType != null) {
+            setHeader(HttpHeaders.Names.CONTENT_TYPE, contentType);
+        }
+
+        this.content = content;
+
+        setHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(content.writerIndex()));
+
+        return this;
+    }
+
+    public Request setBasicAuthentication(String username, String password) {
+        final String base64Value = base64Encode(String.format(Locale.ROOT, "%s:%s", username, password));
+        setHeader(HttpHeaders.Names.AUTHORIZATION, String.format(Locale.ROOT, "Basic %s", base64Value));
+        return this;
+    }
+
+    private String base64Encode(final String value) {
+        final ChannelBufferFactory bufferFactory = HeapChannelBufferFactory.getInstance();
+
+        return Base64
+                .encode(bufferFactory.getBuffer(ByteBuffer.wrap(value.getBytes(Charsets.UTF_8))), false)
+                .toString(Charsets.UTF_8);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ChannelResourceFactory.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ChannelResourceFactory.java
new file mode 100644
index 0000000000..2c46946cdf
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ChannelResourceFactory.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.storage.druid.http.pool;
+
+import com.google.common.base.Preconditions;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+public class ChannelResourceFactory implements ResourceFactory<String, ChannelFuture> {
+    private static final Logger log = LoggerFactory.getLogger(ChannelResourceFactory.class);
+
+    private final ClientBootstrap bootstrap;
+
+    public ChannelResourceFactory(ClientBootstrap bootstrap) {
+        this.bootstrap = Preconditions.checkNotNull(bootstrap, "bootstrap");
+    }
+
+    @Override
+    public ChannelFuture generate(final String hostname) {
+        log.info("Generating: {}", hostname);
+        URL url;
+        try {
+            url = new URL(hostname);
+        } catch (MalformedURLException e) {
+            throw new RuntimeException(e);
+        }
+
+        final String host = url.getHost();
+        final int port = url.getPort() == -1 ? url.getDefaultPort() : url.getPort();
+        return bootstrap.connect(new InetSocketAddress(host, port));
+    }
+
+    @Override
+    public boolean isGood(ChannelFuture resource) {
+        Channel channel = resource.awaitUninterruptibly().getChannel();
+
+        boolean isSuccess = resource.isSuccess();
+        boolean isConnected = channel.isConnected();
+        boolean isOpen = channel.isOpen();
+
+        if (log.isTraceEnabled()) {
+            log.trace("isGood = isSucess[{}] && isConnected[{}] && isOpen[{}]", isSuccess, isConnected, isOpen);
+        }
+
+        return isSuccess && isConnected && isOpen;
+    }
+
+    @Override
+    public void close(ChannelFuture resource) {
+        log.trace("Closing");
+        resource.awaitUninterruptibly().getChannel().close();
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourceContainer.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourceContainer.java
new file mode 100644
index 0000000000..2ca9af8831
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourceContainer.java
@@ -0,0 +1,25 @@
+/*
+ * 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.storage.druid.http.pool;
+
+public interface ResourceContainer<ResourceType> {
+    public ResourceType get();
+
+    public void returnResource();
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourceFactory.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourceFactory.java
new file mode 100644
index 0000000000..8de89000c5
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourceFactory.java
@@ -0,0 +1,27 @@
+/*
+ * 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.storage.druid.http.pool;
+
+public interface ResourceFactory<K, V> {
+    public V generate(K key);
+
+    public boolean isGood(V resource);
+
+    public void close(V resource);
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourcePool.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourcePool.java
new file mode 100644
index 0000000000..60ca2fcf44
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/http/pool/ResourcePool.java
@@ -0,0 +1,239 @@
+/*
+ * 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.storage.druid.http.pool;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.LinkedList;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class ResourcePool<K, V> implements Closeable {
+    private static final Logger log = LoggerFactory.getLogger(ResourcePool.class);
+    private final LoadingCache<K, ImmediateCreationResourceHolder<K, V>> pool;
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public ResourcePool(
+            final ResourceFactory<K, V> factory,
+            final int maxPerKey
+    ) {
+        this.pool = CacheBuilder.newBuilder().build(
+                new CacheLoader<K, ImmediateCreationResourceHolder<K, V>>() {
+                    @Override
+                    public ImmediateCreationResourceHolder<K, V> load(K input) throws Exception {
+                        return new ImmediateCreationResourceHolder<>(
+                                maxPerKey,
+                                input,
+                                factory
+                        );
+                    }
+                }
+        );
+    }
+
+    public ResourceContainer<V> take(final K key) {
+        if (closed.get()) {
+            log.error(String.format(Locale.ROOT, "take(%s) called even though I'm closed.", key));
+            return null;
+        }
+
+        final ImmediateCreationResourceHolder<K, V> holder;
+        try {
+            holder = pool.get(key);
+        } catch (ExecutionException e) {
+            throw Throwables.propagate(e);
+        }
+        final V value = holder.get();
+
+        return new ResourceContainer<V>() {
+            private final AtomicBoolean returned = new AtomicBoolean(false);
+
+            @Override
+            public V get() {
+                Preconditions.checkState(!returned.get(), "Resource for key[%s] has been returned, cannot get().", key);
+                return value;
+            }
+
+            @Override
+            public void returnResource() {
+                if (returned.getAndSet(true)) {
+                    log.warn(String.format(Locale.ROOT, "Resource at key[%s] was returned multiple times?", key));
+                } else {
+                    holder.giveBack(value);
+                }
+            }
+
+            @Override
+            protected void finalize() throws Throwable {
+                if (!returned.get()) {
+                    log.warn(
+                            String.format(Locale.ROOT,
+                                    "Resource[%s] at key[%s] was not returned before Container was finalized, potential resource leak.",
+                                    value,
+                                    key
+                            )
+                    );
+                    returnResource();
+                }
+                super.finalize();
+            }
+        };
+    }
+
+    public void close() {
+        closed.set(true);
+        final Map<K, ImmediateCreationResourceHolder<K, V>> mapView = pool.asMap();
+        for (K k : ImmutableSet.copyOf(mapView.keySet())) {
+            mapView.remove(k).close();
+        }
+    }
+
+    private static class ImmediateCreationResourceHolder<K, V> {
+        private final int maxSize;
+        private final K key;
+        private final ResourceFactory<K, V> factory;
+        private final LinkedList<V> objectList;
+        private int deficit = 0;
+        private boolean closed = false;
+
+        private ImmediateCreationResourceHolder(
+                int maxSize,
+                K key,
+                ResourceFactory<K, V> factory
+        ) {
+            this.maxSize = maxSize;
+            this.key = key;
+            this.factory = factory;
+
+            this.objectList = new LinkedList<V>();
+            for (int i = 0; i < maxSize; ++i) {
+                objectList.addLast(Preconditions.checkNotNull(factory.generate(key), "factory.generate(key)"));
+            }
+        }
+
+        V get() {
+            // objectList can't have nulls, so we'll use a null to signal that we need to create a new resource.
+            final V poolVal;
+            synchronized (this) {
+                while (!closed && objectList.size() == 0 && deficit == 0) {
+                    try {
+                        this.wait();
+                    } catch (InterruptedException e) {
+                        Thread.interrupted();
+                        return null;
+                    }
+                }
+
+                if (closed) {
+                    log.info(String.format(Locale.ROOT, "get() called even though I'm closed. key[%s]", key));
+                    return null;
+                } else if (!objectList.isEmpty()) {
+                    poolVal = objectList.removeFirst();
+                } else if (deficit > 0) {
+                    deficit--;
+                    poolVal = null;
+                } else {
+                    throw new IllegalStateException("WTF?! No objects left, and no object deficit. This is probably a bug.");
+                }
+            }
+
+            // At this point, we must either return a valid resource or increment "deficit".
+            final V retVal;
+            try {
+                if (poolVal != null && factory.isGood(poolVal)) {
+                    retVal = poolVal;
+                } else {
+                    if (poolVal != null) {
+                        factory.close(poolVal);
+                    }
+                    retVal = factory.generate(key);
+                }
+            } catch (Throwable e) {
+                synchronized (this) {
+                    deficit++;
+                    this.notifyAll();
+                }
+                throw Throwables.propagate(e);
+            }
+
+            return retVal;
+        }
+
+        void giveBack(V object) {
+            Preconditions.checkNotNull(object, "object");
+
+            synchronized (this) {
+                if (closed) {
+                    log.info(String.format(Locale.ROOT, "giveBack called after being closed. key[%s]", key));
+                    factory.close(object);
+                    return;
+                }
+
+                if (objectList.size() >= maxSize) {
+                    if (objectList.contains(object)) {
+                        log.warn(
+                                String.format(
+                                        Locale.ROOT,
+                                        "Returning object[%s] at key[%s] that has already been returned!? Skipping",
+                                        object,
+                                        key
+                                ),
+                                new Exception("Exception for stacktrace")
+                        );
+                    } else {
+                        log.warn(
+                                String.format(
+                                        Locale.ROOT,
+                                        "Returning object[%s] at key[%s] even though we already have all that we can hold[%s]!? Skipping",
+                                        object,
+                                        key,
+                                        objectList
+                                ),
+                                new Exception("Exception for stacktrace")
+                        );
+                    }
+                    return;
+                }
+
+                objectList.addLast(object);
+                this.notifyAll();
+            }
+        }
+
+        void close() {
+            synchronized (this) {
+                closed = true;
+                while (!objectList.isEmpty()) {
+                    factory.close(objectList.removeFirst());
+                }
+                this.notifyAll();
+            }
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/ColumnFiller.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/ColumnFiller.java
new file mode 100644
index 0000000000..1e346493e6
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/ColumnFiller.java
@@ -0,0 +1,25 @@
+/*
+ * 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.storage.druid.read;
+
+import org.apache.kylin.metadata.tuple.Tuple;
+
+public interface ColumnFiller {
+    void fill(Object[] row, Tuple tuple);
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DerivedColumnFiller.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DerivedColumnFiller.java
new file mode 100644
index 0000000000..52b47918d5
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DerivedColumnFiller.java
@@ -0,0 +1,55 @@
+/*
+ * 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.storage.druid.read;
+
+import java.util.Objects;
+
+import org.apache.kylin.common.util.Array;
+import org.apache.kylin.dict.lookup.ILookupTable;
+import org.apache.kylin.metadata.tuple.Tuple;
+
+public class DerivedColumnFiller implements ColumnFiller {
+    private final DerivedIndexMapping mapping;
+    private final ILookupTable lookupTable;
+    private final int numHostCols;
+    private final int numDerivedCols;
+    private final Array<String> key;
+
+    DerivedColumnFiller(DerivedIndexMapping mapping, ILookupTable lookupTable) {
+        this.mapping = mapping;
+        this.lookupTable = lookupTable;
+        this.numHostCols = mapping.numHostColumns();
+        this.numDerivedCols = mapping.numDerivedColumns();
+        this.key = new Array<>(new String[numHostCols]);
+    }
+
+    @Override
+    public void fill(Object[] row, Tuple tuple) {
+        for (int i = 0; i < numHostCols; i++) {
+            key.data[i] = Objects.toString(row[mapping.getHostIndex(i)], null);
+        }
+        String[] lookupRow = lookupTable.getRow(key);
+        for (int i = 0; i < numDerivedCols; i++) {
+            if (mapping.getTupleIndex(i) >= 0) {
+                String value = lookupRow == null ? null : lookupRow[mapping.getLookupIndex(i)];
+                tuple.setDimensionValue(mapping.getTupleIndex(i), value);
+            }
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DerivedIndexMapping.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DerivedIndexMapping.java
new file mode 100644
index 0000000000..f56e080f69
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DerivedIndexMapping.java
@@ -0,0 +1,88 @@
+/*
+ * 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.storage.druid.read;
+
+import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.tuple.TupleInfo;
+import org.apache.kylin.storage.druid.DruidSchema;
+
+public class DerivedIndexMapping {
+    private final int[] tupleIndex;
+    private final int[] lookupIndex;
+    private final int[] hostIndex;
+
+    public DerivedIndexMapping(DeriveInfo info, DruidSchema schema, TupleInfo tupleInfo) {
+        TblColRef[] derivedCols = info.columns;
+        TblColRef[] hostCols = info.join.getForeignKeyColumns();
+
+        this.tupleIndex = new int[derivedCols.length];
+        this.lookupIndex = new int[derivedCols.length];
+        this.hostIndex = new int[hostCols.length];
+
+        for (int i = 0; i < derivedCols.length; i++) {
+            TblColRef col = derivedCols[i];
+            tupleIndex[i] = tupleInfo.hasColumn(col) ? tupleInfo.getColumnIndex(col) : -1;
+            lookupIndex[i] = col.getColumnDesc().getZeroBasedIndex();
+        }
+
+        for (int i = 0; i < hostCols.length; i++) {
+            hostIndex[i] = schema.getDimensions().indexOf(hostCols[i]);
+        }
+    }
+
+    public boolean shouldReturnDerived() {
+        for (int i = 0; i < tupleIndex.length; i++) {
+            if (tupleIndex[i] == -1) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public int numHostColumns() {
+        return hostIndex.length;
+    }
+
+    public int numDerivedColumns() {
+        return tupleIndex.length;
+    }
+
+    /**
+     * @param i [0, {@link #numHostColumns()})
+     * @return row index of the i-th host column
+     */
+    public int getHostIndex(int i) {
+        return hostIndex[i];
+    }
+
+    /**
+     * @return lookup table's index containing the i-th derived column
+     */
+    public int getLookupIndex(int i) {
+        return lookupIndex[i];
+    }
+
+    /**
+     * @return tuple index for the i-th derived column
+     */
+    public int getTupleIndex(int i) {
+        return tupleIndex[i];
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidColumnFiller.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidColumnFiller.java
new file mode 100644
index 0000000000..3a058deaa5
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidColumnFiller.java
@@ -0,0 +1,87 @@
+/*
+ * 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.storage.druid.read;
+
+import java.util.Objects;
+
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.TupleInfo;
+import org.apache.kylin.storage.druid.DruidSchema;
+
+public class DruidColumnFiller implements ColumnFiller {
+    private final DruidSchema schema;
+
+    private final int[] tupleIndex;
+    private final MeasureType[] measureTypes;
+
+    public DruidColumnFiller(DruidSchema schema, TupleInfo tupleInfo) {
+        this.schema = schema;
+        this.tupleIndex = new int[schema.getTotalFieldCount()];
+        this.measureTypes = new MeasureType[schema.getTotalFieldCount()];
+
+        int i = 0;
+        for (TblColRef dim : schema.getDimensions()) {
+            tupleIndex[i++] = tupleInfo.getColumnIndex(dim);
+        }
+        for (MeasureDesc met : schema.getMeasures()) {
+            FunctionDesc func = met.getFunction();
+            MeasureType<?> measureType = func.getMeasureType();
+
+            if (func.needRewrite()) {
+                String fieldName = func.getRewriteFieldName();
+                tupleIndex[i] = tupleInfo.getFieldIndex(fieldName);
+            } else {
+                TblColRef col = func.getParameter().getColRefs().get(0);
+                tupleIndex[i] = tupleInfo.getColumnIndex(col);
+            }
+
+            if (!measureType.needAdvancedTupleFilling()) {
+                measureTypes[i] = measureType;
+            } else if (measureType instanceof ExtendedColumnMeasureType) {
+                final TblColRef extended = ExtendedColumnMeasureType.getExtendedColumn(func);
+                final int extendedColumnInTupleIdx = tupleInfo.hasColumn(extended) ? tupleInfo.getColumnIndex(extended) : -1;
+                tupleIndex[i] = extendedColumnInTupleIdx;
+            } else {
+                throw new UnsupportedOperationException("Unsupported measure type : " + measureType);
+            }
+
+            i++;
+        }
+    }
+
+    public void fill(Object[] row, Tuple tuple) {
+        for (int i = 0; i < schema.getDimensions().size(); i++) {
+            tuple.setDimensionValue(tupleIndex[i], Objects.toString(row[i], null));
+        }
+
+        for (int i = schema.getDimensions().size(); i < schema.getTotalFieldCount(); i++) {
+            if (measureTypes[i] != null) {
+                measureTypes[i].fillTupleSimply(tuple, tupleIndex[i], row[i]);
+            } else {
+                //for ExtendedColumn
+                tuple.setDimensionValue(tupleIndex[i], (String) row[i]);
+            }
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidStorageQuery.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidStorageQuery.java
new file mode 100644
index 0000000000..23c25b9546
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidStorageQuery.java
@@ -0,0 +1,332 @@
+/*
+ * 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.storage.druid.read;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.RawQueryLastHacker;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilterVisitor2;
+import org.apache.kylin.metadata.filter.TupleFilterVisitor2Adaptor;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.metadata.tuple.TupleInfo;
+import org.apache.kylin.storage.IStorageQuery;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.NameMapping;
+import org.apache.kylin.storage.druid.NameMappingFactory;
+import org.apache.kylin.storage.druid.read.cursor.RowCursor;
+import org.apache.kylin.storage.druid.read.cursor.RowCursorFactory;
+import org.apache.kylin.storage.druid.read.filter.FilterCondition;
+import org.apache.kylin.storage.druid.read.filter.MoreDimFilters;
+import org.apache.kylin.storage.druid.read.filter.PruneIntervalsProcessor;
+import org.apache.kylin.storage.druid.read.filter.RemoveMarkerProcessor;
+import org.apache.kylin.storage.druid.read.filter.ToDimFilterVisitor;
+import org.apache.kylin.storage.druid.read.filter.TranslateDerivedVisitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import io.druid.java.util.common.granularity.Granularity;
+import io.druid.query.DataSource;
+import io.druid.query.Query;
+import io.druid.query.TableDataSource;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.DimFilters;
+import io.druid.query.groupby.GroupByQuery;
+import io.druid.query.scan.ScanQuery;
+import io.druid.query.spec.MultipleIntervalSegmentSpec;
+import io.druid.query.spec.QuerySegmentSpec;
+
+public class DruidStorageQuery implements IStorageQuery {
+    private static final Logger logger = LoggerFactory.getLogger(DruidStorageQuery.class);
+
+    private final CubeInstance cubeInstance;
+    private final CubeDesc cubeDesc;
+    private final RowCursorFactory cursorFactory;
+    private final NameMapping nameMapping;
+
+    public DruidStorageQuery(CubeInstance cube, RowCursorFactory cursorFactory) {
+        this.cubeInstance = cube;
+        this.cubeDesc = cube.getDescriptor();
+        this.cursorFactory = cursorFactory;
+        this.nameMapping = NameMappingFactory.getDefault(cubeDesc);
+    }
+
+    @Override
+    public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
+        context.setStorageQuery(this);
+        //context.getQueryContext().setDeadline(cubeInstance.getConfig().getQueryTimeoutSeconds() * 1000);
+
+        final LookupTableCache lookupCache = new LookupTableCache(cubeInstance);
+
+        final StorageRequest request = planRequest(context, sqlDigest, returnTupleInfo, lookupCache);
+
+        // convert to druid filter
+        List<DimFilter> dimFilters = Lists.newArrayList();
+        dimFilters.add(DimFilters.dimEquals(DruidSchema.ID_COL, String.valueOf(request.cuboid.getId())));
+        if (request.filter != null) {
+            ToDimFilterVisitor dimFilterVisitor = new ToDimFilterVisitor(nameMapping, request.dimensions);
+            DimFilter dimFilter = request.filter.accept(new TupleFilterVisitor2Adaptor<>(dimFilterVisitor));
+            dimFilters.add(dimFilter);
+        }
+        DimFilter dimFilter = MoreDimFilters.and(dimFilters);
+
+        // optimize filter condition
+        FilterCondition condition = FilterCondition.of(ImmutableList.of(DruidSchema.ETERNITY_INTERVAL), dimFilter);
+        condition = optimizeFilter(condition);
+        if (condition.getIntervals().isEmpty()) {
+            return ITupleIterator.EMPTY_TUPLE_ITERATOR;
+        }
+
+        DruidSchema schema = new DruidSchema(nameMapping, request.groups, request.measures);
+        Query<?> query = makeDruidQuery(DruidSchema.getDataSource(cubeDesc), schema, condition, context);
+        RowCursor cursor = cursorFactory.createRowCursor(schema, query, context);
+        return new DruidTupleIterator(cubeInstance, lookupCache, schema, returnTupleInfo, cursor, context);
+    }
+
+    protected StorageRequest planRequest(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo, LookupTableCache lookupCache) {
+        //deal with participant columns in subquery join
+        sqlDigest.includeSubqueryJoinParticipants();
+
+        //cope with queries with no aggregations
+        RawQueryLastHacker.hackNoAggregations(sqlDigest, cubeDesc, returnTupleInfo);
+
+        // Customized measure taking effect: e.g. allow custom measures to help raw queries
+        notifyBeforeStorageQuery(sqlDigest);
+
+        TupleFilter filter = sqlDigest.filter;
+        List<MeasureDesc> measures = findMeasures(sqlDigest);
+
+        // expand derived (xxxD means contains host columns only, derived columns were translated)
+        Set<TblColRef> groupsD = translateDerivedToHost(sqlDigest.groupbyColumns);
+        Set<TblColRef> filterD = Sets.newHashSet();
+        if (filter != null) {
+            TupleFilterVisitor2<TupleFilter> translateDerivedVisitor = new TranslateDerivedVisitor(cubeDesc, lookupCache);
+            filter = filter.accept(new TupleFilterVisitor2Adaptor<>(translateDerivedVisitor));
+            TupleFilter.collectColumns(filter, filterD);
+        }
+
+        // identify cuboid
+        Set<TblColRef> dimensionsD = Sets.newLinkedHashSet();
+        dimensionsD.addAll(groupsD);
+        dimensionsD.addAll(filterD);
+        Cuboid cuboid = Cuboid.findCuboid(cubeInstance.getCuboidScheduler(), dimensionsD, toFunctions(measures));
+        context.setCuboid(cuboid);
+
+        // determine whether push down aggregation to storage is beneficial
+        Set<TblColRef> singleValuesD = findSingleValueColumns(filter);
+        context.setNeedStorageAggregation(needStorageAggregation(cuboid, groupsD, singleValuesD));
+
+        logger.info("Cuboid identified: cube={}, cuboidId={}, groupsD={}, filterD={}, limitPushdown={}, storageAggr={}", cubeInstance.getName(), cuboid.getId(), groupsD, filterD, context.getFinalPushDownLimit(), context.isNeedStorageAggregation());
+        return new StorageRequest(cuboid, Lists.newArrayList(dimensionsD), Lists.newArrayList(groupsD), measures, filter);
+    }
+
+    private void notifyBeforeStorageQuery(SQLDigest sqlDigest) {
+        Map<String, List<MeasureDesc>> map = Maps.newHashMap();
+        for (MeasureDesc measure : cubeDesc.getMeasures()) {
+            MeasureType<?> measureType = measure.getFunction().getMeasureType();
+
+            String key = measureType.getClass().getCanonicalName();
+            List<MeasureDesc> temp = null;
+            if ((temp = map.get(key)) != null) {
+                temp.add(measure);
+            } else {
+                map.put(key, Lists.<MeasureDesc> newArrayList(measure));
+            }
+        }
+
+        for (List<MeasureDesc> sublist : map.values()) {
+            sublist.get(0).getFunction().getMeasureType().adjustSqlDigest(sublist, sqlDigest);
+        }
+    }
+
+    private List<MeasureDesc> findMeasures(SQLDigest sqlDigest) {
+        List<MeasureDesc> result = Lists.newArrayList();
+        for (MeasureDesc measure : cubeDesc.getMeasures()) {
+            if (sqlDigest.aggregations.contains(measure.getFunction())) {
+                result.add(measure);
+            }
+        }
+        return result;
+    }
+
+    private List<FunctionDesc> toFunctions(Collection<MeasureDesc> measures) {
+        List<FunctionDesc> result = Lists.newArrayListWithExpectedSize(measures.size());
+        for (MeasureDesc measure : measures) {
+            result.add(measure.getFunction());
+        }
+        return result;
+    }
+
+    private Set<TblColRef> translateDerivedToHost(Collection<TblColRef> columns) {
+        Set<TblColRef> result = Sets.newLinkedHashSet();
+        for (TblColRef column : columns) {
+            if (cubeDesc.hasHostColumn(column)) {
+                CubeDesc.DeriveInfo hostInfo = cubeDesc.getHostInfo(column);
+                Collections.addAll(result, hostInfo.columns);
+            } else {
+                result.add(column);
+            }
+        }
+        return result;
+    }
+
+    private Set<TblColRef> findSingleValueColumns(TupleFilter filter) {
+        Set<TblColRef> result = Sets.newHashSet();
+
+        List<CompareTupleFilter> toCheck = Lists.newArrayList();
+        if (filter instanceof CompareTupleFilter) {
+            toCheck.add((CompareTupleFilter) filter);
+        }
+        if (filter instanceof LogicalTupleFilter && filter.getOperator() == TupleFilter.FilterOperatorEnum.AND) {
+            for (TupleFilter child : filter.getChildren()) {
+                if (child instanceof CompareTupleFilter) {
+                    toCheck.add((CompareTupleFilter) child);
+                }
+            }
+        }
+
+        for (CompareTupleFilter compare : toCheck) {
+            if (compare.columnMatchSingleValue()) {
+                result.add(compare.getColumn());
+            }
+        }
+        return result;
+    }
+
+    private boolean needStorageAggregation(Cuboid cuboid, Collection<TblColRef> groupD, Collection<TblColRef> singleValueD) {
+        HashSet<TblColRef> temp = Sets.newHashSet();
+        temp.addAll(groupD);
+        temp.addAll(singleValueD);
+        return cuboid.getColumns().size() != temp.size();
+    }
+
+    @Override
+    public boolean keepRuntimeFilter() {
+        return false;
+    }
+
+    private FilterCondition optimizeFilter(FilterCondition condition) {
+        List<Function<FilterCondition, FilterCondition>> processors = ImmutableList.of(
+                RemoveMarkerProcessor.SINGLETON,
+                new PruneIntervalsProcessor(cubeInstance, nameMapping)
+        );
+
+        for (Function<FilterCondition, FilterCondition> processor : processors) {
+            condition = processor.apply(condition);
+        }
+        return condition;
+    }
+
+    private Query<?> makeDruidQuery(String dataSource, DruidSchema schema, FilterCondition condition, StorageContext context) {
+        Map<String, Object> druidContext = new HashMap<>();
+        druidContext.put("queryId", context.getQueryContext().getQueryId());
+        // scan query has trouble with very large timeout value, so we only set druid timeout
+        // when user configures a non-zero kylin timeout
+//        if (context.getQueryContext().getDeadline() != Long.MAX_VALUE) {
+//            druidContext.put(QueryContexts.TIMEOUT_KEY, context.getQueryContext().checkMillisBeforeDeadline());
+//        }
+
+        final Query<?> query;
+        final DataSource ds = new TableDataSource(dataSource);
+        final QuerySegmentSpec segmentSpec = new MultipleIntervalSegmentSpec(condition.getIntervals());
+
+        if (context.isNeedStorageAggregation()) {
+            druidContext.put("finalize", false);
+            query = new GroupByQuery(
+                    ds, segmentSpec, null,
+                    condition.getDimFilter(),
+                    Granularity.fromString("all"),
+                    schema.getQueryDimensionSpec(schema.getDimensions()),
+                    Lists.newArrayList(schema.getAggregators()),
+                    null, null, null,
+                    druidContext
+            );
+
+        } else {
+            boolean hasMemoryHungryMetric = false;
+            for (MeasureDesc metric : schema.getMeasures()) {
+                hasMemoryHungryMetric |= metric.getFunction().getMeasureType().isMemoryHungry();
+            }
+            int batchSize = hasMemoryHungryMetric ? 10 : 1024;
+
+            List<String> columns = new ArrayList<>();
+            for (TblColRef dim : schema.getDimensions()) {
+                columns.add(schema.getDimFieldName(dim));
+            }
+            for (MeasureDesc met : schema.getMeasures()) {
+                columns.add(schema.getMeasureFieldName(met));
+            }
+
+            query = new ScanQuery(
+                    ds,
+                    segmentSpec,
+                    null,
+                    ScanQuery.RESULT_FORMAT_COMPACTED_LIST,
+                    batchSize,
+                    0,
+                    condition.getDimFilter(),
+                    columns,
+                    false,
+                    druidContext
+            );
+        }
+        return query;
+    }
+
+    private static class StorageRequest {
+        final Cuboid cuboid;
+        final List<TblColRef> dimensions;
+        final List<TblColRef> groups;
+        final List<MeasureDesc> measures;
+        final TupleFilter filter;
+
+        public StorageRequest(Cuboid cuboid, List<TblColRef> dimensions, List<TblColRef> groups, List<MeasureDesc> measures, TupleFilter filter) {
+            this.cuboid = cuboid;
+            this.dimensions = dimensions;
+            this.groups = groups;
+            this.measures = measures;
+            this.filter = filter;
+        }
+    }
+
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidTupleIterator.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidTupleIterator.java
new file mode 100644
index 0000000000..8a4e8ad797
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/DruidTupleIterator.java
@@ -0,0 +1,110 @@
+/*
+ * 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.storage.druid.read;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
+import org.apache.kylin.dict.lookup.ILookupTable;
+import org.apache.kylin.metadata.tuple.ITuple;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.metadata.tuple.Tuple;
+import org.apache.kylin.metadata.tuple.TupleInfo;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.read.cursor.RowCursor;
+
+public class DruidTupleIterator implements ITupleIterator {
+    private final List<ColumnFiller> fillers;
+    private final RowCursor cursor;
+    private final Tuple tuple;
+    private final StorageContext context;
+
+    private int scanCount;
+
+    public DruidTupleIterator(CubeInstance cube, LookupTableCache lookupCache, DruidSchema schema, TupleInfo tupleInfo, RowCursor cursor, StorageContext context) {
+        this.cursor = cursor;
+        this.tuple = new Tuple(tupleInfo);
+        this.context = context;
+
+        this.fillers = new ArrayList<>();
+        fillers.add(new DruidColumnFiller(schema, tupleInfo));
+
+        // derived columns fillers
+        List<DeriveInfo> deriveInfos = cube.getDescriptor().getDeriveInfos(schema.getDimensions());
+
+        for (DeriveInfo info : deriveInfos) {
+            DerivedIndexMapping mapping = new DerivedIndexMapping(info, schema, tupleInfo);
+            if (!mapping.shouldReturnDerived()) {
+                continue;
+            }
+            if (info.type == CubeDesc.DeriveType.LOOKUP) {
+                ILookupTable lookupTable = lookupCache.get(info.join);
+                fillers.add(new DerivedColumnFiller(mapping, lookupTable));
+
+            } else if (info.type == CubeDesc.DeriveType.PK_FK) {
+                fillers.add(new PkColumnFiller(mapping));
+            }
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        return cursor.hasNext();
+    }
+
+    @Override
+    public ITuple next() {
+        if (!hasNext()) {
+            throw new NoSuchElementException();
+        }
+
+        if (scanCount++ >= 100) {
+            flushScanCountDelta();
+            context.getQueryContext().checkMillisBeforeDeadline();
+        }
+
+        Object[] row = cursor.next();
+        for (ColumnFiller filler : fillers) {
+            filler.fill(row, tuple);
+        }
+        return tuple;
+    }
+
+    @Override
+    public void close() {
+        flushScanCountDelta();
+        IOUtils.closeQuietly(cursor);
+    }
+
+    private void flushScanCountDelta() {
+        context.increaseProcessedRowCount(scanCount);
+        scanCount = 0;
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException("remove");
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/LookupTableCache.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/LookupTableCache.java
new file mode 100644
index 0000000000..41d14ffba2
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/LookupTableCache.java
@@ -0,0 +1,51 @@
+/*
+ * 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.storage.druid.read;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.dict.lookup.ILookupTable;
+import org.apache.kylin.metadata.model.JoinDesc;
+
+// not thread-safe
+public class LookupTableCache {
+    private CubeSegment segment;
+    private CubeManager manager;
+
+    private Map<String, ILookupTable> cache = new HashMap<>();
+
+    public LookupTableCache(CubeInstance cube) {
+        this.segment = cube.getLatestReadySegment();
+        this.manager = CubeManager.getInstance(cube.getConfig());
+    }
+
+    public ILookupTable get(JoinDesc joinDesc) {
+        String tableIdentity = joinDesc.getPKSide().getTableIdentity();
+        ILookupTable result = cache.get(tableIdentity);
+        if (result == null) {
+            result = manager.getLookupTable(segment, joinDesc);
+            cache.put(tableIdentity, result);
+        }
+        return result;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/PkColumnFiller.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/PkColumnFiller.java
new file mode 100644
index 0000000000..74153778d6
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/PkColumnFiller.java
@@ -0,0 +1,37 @@
+/*
+ * 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.storage.druid.read;
+
+import java.util.Objects;
+
+import org.apache.kylin.metadata.tuple.Tuple;
+
+public class PkColumnFiller implements ColumnFiller {
+    private final DerivedIndexMapping mapping;
+
+    PkColumnFiller(DerivedIndexMapping mapping) {
+        this.mapping = mapping;
+    }
+
+    @Override
+    public void fill(Object[] row, Tuple tuple) {
+        Object value = row[mapping.getHostIndex(0)];
+        tuple.setDimensionValue(mapping.getTupleIndex(0), Objects.toString(value, null));
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/AbstractRowCursor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/AbstractRowCursor.java
new file mode 100644
index 0000000000..e7a169bbf9
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/AbstractRowCursor.java
@@ -0,0 +1,112 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import org.apache.kylin.measure.bitmap.RoaringBitmapCounterFactory;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.DruidSchema;
+
+import io.druid.collections.bitmap.WrappedImmutableRoaringBitmap;
+import io.druid.query.Query;
+
+public abstract class AbstractRowCursor<T> implements RowCursor {
+    protected final DruidSchema schema;
+    protected final DruidClient<T> client;
+    protected final Query<T> query;
+    protected final StorageContext context;
+
+    private final Object[] row;
+    private boolean fetched; // indicate whether the next row is fetched
+
+    private Cursor<T> resultCursor;
+    private T result;
+    private int nextRow;
+
+    public AbstractRowCursor(DruidSchema schema, DruidClient<T> client, Query<T> query, StorageContext context) {
+        this.schema = schema;
+        this.client = client;
+        this.query = query;
+        this.context = context;
+        this.row = new Object[schema.getTotalFieldCount()];
+    }
+
+    /**
+     * Convert the i-th value in `result` to `rows`.
+     * @return true if i-th row exists, false otherwise
+     */
+    protected abstract boolean convert(T result, int index, Object[] row);
+
+    protected Object convertComplexValue(Object value) {
+        if (value instanceof WrappedImmutableRoaringBitmap) {
+            WrappedImmutableRoaringBitmap bitmap = (WrappedImmutableRoaringBitmap) value;
+            return RoaringBitmapCounterFactory.INSTANCE.newBitmap(bitmap.getBitmap());
+        }
+        return value;
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (fetched) {
+            return true;
+        }
+
+        if (resultCursor == null) {
+            resultCursor = client.execute(query, context);
+        }
+
+        while (result != null || resultCursor.hasNext()) {
+            if (result == null) {
+                result = resultCursor.next();
+            }
+            fetched = convert(result, nextRow++, row);
+            if (fetched) {
+                return true;
+            }
+            // try next result
+            result = null;
+            nextRow = 0;
+        }
+
+        return false;
+    }
+
+    @Override
+    public Object[] next() {
+        if (!hasNext()) {
+            throw new NoSuchElementException();
+        }
+        fetched = false;
+        return row;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (resultCursor != null) {
+            resultCursor.close();
+        }
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException("remove");
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/Cursor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/Cursor.java
new file mode 100644
index 0000000000..254db402d6
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/Cursor.java
@@ -0,0 +1,25 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import java.io.Closeable;
+import java.util.Iterator;
+
+public interface Cursor<T> extends Iterator<T>, Closeable {
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/DruidClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/DruidClient.java
new file mode 100644
index 0000000000..a6f5092180
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/DruidClient.java
@@ -0,0 +1,31 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import org.apache.kylin.storage.StorageContext;
+
+import io.druid.query.Query;
+
+/**
+ *
+ * @param <T> result row type
+ */
+public interface DruidClient<T> {
+    Cursor<T> execute(Query<T> query, StorageContext context);
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/GroupByRowCursor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/GroupByRowCursor.java
new file mode 100644
index 0000000000..67de596a2d
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/GroupByRowCursor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import java.util.List;
+
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.DruidSchema;
+
+import io.druid.data.input.Row;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.groupby.GroupByQuery;
+
+public class GroupByRowCursor extends AbstractRowCursor<Row> {
+    private final List<AggregatorFactory> factories;
+
+    public GroupByRowCursor(DruidSchema schema, DruidClient<Row> client, GroupByQuery query, StorageContext context) {
+        super(schema, client, query, context);
+        this.factories = query.getAggregatorSpecs();
+    }
+
+    @Override
+    protected boolean convert(Row result, int index, Object[] row) {
+        if (index > 0) {
+            return false;
+        }
+
+        for (TblColRef dim : schema.getDimensions()) {
+            String name = schema.getDimFieldName(dim);
+            row[schema.getFieldIndex(name)] = result.getRaw(name);
+        }
+
+        for (AggregatorFactory factory : factories) {
+            String name = factory.getName();
+            Object object = factory.deserialize(result.getRaw(name));
+            row[schema.getFieldIndex(name)] = convertComplexValue(object);
+        }
+        return true;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/HttpDruidClient.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/HttpDruidClient.java
new file mode 100644
index 0000000000..6bf9d5d5d4
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/HttpDruidClient.java
@@ -0,0 +1,301 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.core.ObjectCodec;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.smile.SmileFactory;
+import com.google.common.base.Throwables;
+import com.google.common.io.ByteSource;
+import com.google.common.io.ByteStreams;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.RE;
+import io.druid.java.util.common.guava.CloseQuietly;
+import io.druid.query.Query;
+import io.druid.query.QueryInterruptedException;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.http.ClientResponse;
+import org.apache.kylin.storage.druid.http.HttpClient;
+import org.apache.kylin.storage.druid.http.HttpResponseHandler;
+import org.apache.kylin.storage.druid.http.Request;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBufferInputStream;
+import org.jboss.netty.handler.codec.http.HttpChunk;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.SequenceInputStream;
+import java.net.URL;
+import java.util.Enumeration;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class HttpDruidClient<T> implements DruidClient<T> {
+    private static final Logger logger = LoggerFactory.getLogger(HttpDruidClient.class);
+
+    private final HttpClient httpClient;
+    private final ObjectMapper objectMapper;
+    private final TypeReference<T> typeReference;
+    private final String host;
+    private final boolean isSmile;
+
+    public HttpDruidClient(HttpClient httpClient, ObjectMapper objectMapper, TypeReference<T> typeReference, String host) {
+        this.httpClient = httpClient;
+        this.objectMapper = objectMapper;
+        this.typeReference = typeReference;
+        this.host = host;
+        this.isSmile = this.objectMapper.getFactory() instanceof SmileFactory;
+    }
+
+    @SuppressWarnings("checkstyle:methodlength")
+    @Override
+    public Cursor<T> execute(final Query<T> query, final StorageContext context) {
+        final ListenableFuture<InputStream> future;
+        final String url = String.format(Locale.ROOT, "http://%s/druid/v2/", host);
+        final long requestStartTime = System.currentTimeMillis();
+
+        final HttpResponseHandler<InputStream, InputStream> responseHandler = new HttpResponseHandler<InputStream, InputStream>() {
+            private final AtomicLong byteCount = new AtomicLong(0);
+            private final BlockingQueue<InputStream> queue = new LinkedBlockingQueue<>();
+            private final AtomicBoolean done = new AtomicBoolean(false);
+            private long responseStartTime;
+
+            @Override
+            public ClientResponse<InputStream> handleResponse(HttpResponse response) {
+                responseStartTime = System.currentTimeMillis();
+
+                try {
+                    queue.put(new ChannelBufferInputStream(response.getContent()));
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw Throwables.propagate(e);
+                }
+                byteCount.addAndGet(response.getContent().readableBytes());
+                return ClientResponse.<InputStream>finished(
+                        new SequenceInputStream(
+                                new Enumeration<InputStream>() {
+                                    @Override
+                                    public boolean hasMoreElements() {
+                                        // Done is always true until the last stream has be put in the queue.
+                                        // Then the stream should be spouting good InputStreams.
+                                        synchronized (done) {
+                                            return !done.get() || !queue.isEmpty();
+                                        }
+                                    }
+
+                                    @Override
+                                    public InputStream nextElement() {
+                                        try {
+                                            return queue.take();
+                                        } catch (InterruptedException e) {
+                                            Thread.currentThread().interrupt();
+                                            throw Throwables.propagate(e);
+                                        }
+                                    }
+                                }
+                        )
+                );
+            }
+
+            @Override
+            public ClientResponse<InputStream> handleChunk(
+                    ClientResponse<InputStream> clientResponse, HttpChunk chunk
+            ) {
+                final ChannelBuffer channelBuffer = chunk.getContent();
+                final int bytes = channelBuffer.readableBytes();
+                if (bytes > 0) {
+                    try {
+                        queue.put(new ChannelBufferInputStream(channelBuffer));
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                        throw Throwables.propagate(e);
+                    }
+                    byteCount.addAndGet(bytes);
+                }
+                return clientResponse;
+            }
+
+            @Override
+            public ClientResponse<InputStream> done(ClientResponse<InputStream> clientResponse) {
+                // FIXME collect druid scan bytes instead of http response size
+                context.getQueryContext().addAndGetScannedBytes(byteCount.get());
+
+                logger.debug("Completed request for queryId={} intervals={} bytes={} ttfb={} time={}",
+                        query.getId(),
+                        query.getIntervals(),
+                        byteCount.get(),
+                        responseStartTime - requestStartTime,
+                        System.currentTimeMillis() - requestStartTime);
+
+
+                synchronized (done) {
+                    try {
+                        // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out
+                        // after done is set to true, regardless of the rest of the stream's state.
+                        ByteSource empty = ByteStreams.asByteSource(new byte[]{});
+                        queue.put(empty.openStream());
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                        throw Throwables.propagate(e);
+                    } catch (IOException e) {
+                        // This should never happen
+                        throw Throwables.propagate(e);
+                    } finally {
+                        done.set(true);
+                    }
+                }
+                return ClientResponse.finished(clientResponse.getObj());
+            }
+
+            @Override
+            public void exceptionCaught(final ClientResponse<InputStream> clientResponse, final Throwable e) {
+                // FIXME collect druid scan bytes instead of http response size
+                context.getQueryContext().addAndGetScannedBytes(byteCount.get());
+
+                logger.debug("Failed request for queryId={} intervals={} bytes={} exception={}",
+                        query.getId(),
+                        query.getIntervals(),
+                        byteCount.get(),
+                        e.getMessage());
+
+                // Don't wait for lock in case the lock had something to do with the error
+                synchronized (done) {
+                    done.set(true);
+                    // Make a best effort to put a zero length buffer into the queue in case something is waiting on the take()
+                    // If nothing is waiting on take(), this will be closed out anyways.
+                    queue.offer(
+                            new InputStream() {
+                                @Override
+                                public int read() throws IOException {
+                                    throw new IOException(e);
+                                }
+                            }
+                    );
+                }
+            }
+        };
+
+        try {
+            future = httpClient.go(
+                    new Request(
+                            HttpMethod.POST,
+                            new URL(url)
+                    ).setContent(objectMapper.writeValueAsBytes(query))
+                            .setHeader(
+                                    HttpHeaders.Names.CONTENT_TYPE,
+                                    isSmile ? "application/x-jackson-smile" : "application/json"
+                            ),
+                    responseHandler
+            );
+
+        } catch (IOException e) {
+            throw Throwables.propagate(e);
+        }
+
+        return new Cursor<T>() {
+            private JsonParser jp;
+            private ObjectCodec objectCodec;
+            private T next;
+
+            private void init() {
+                if (jp == null) {
+                    try {
+                        jp = objectMapper.getFactory().createParser(future.get());
+                        final JsonToken nextToken = jp.nextToken();
+                        if (nextToken == JsonToken.START_OBJECT) {
+                            QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class);
+                            throw new QueryInterruptedException(cause, host);
+                        } else if (nextToken != JsonToken.START_ARRAY) {
+                            throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url);
+                        } else {
+                            jp.nextToken();
+                            objectCodec = jp.getCodec();
+                        }
+                    } catch (IOException | InterruptedException | ExecutionException e) {
+                        throw new RE(e, "Failure getting results from[%s] because of [%s]", url, e.getMessage());
+                    } catch (CancellationException e) {
+                        throw Throwables.propagate(e);
+                    }
+                }
+            }
+
+            @Override
+            public void close() throws IOException {
+                if (jp != null) {
+                    jp.close();
+                }
+            }
+
+            @Override
+            public boolean hasNext() {
+                init();
+
+                if (next != null) {
+                    return true;
+                }
+
+                if (jp.isClosed()) {
+                    return false;
+                }
+                if (jp.getCurrentToken() == JsonToken.END_ARRAY) {
+                    CloseQuietly.close(jp);
+                    return false;
+                }
+
+                try {
+                    next = objectCodec.readValue(jp, typeReference);
+                    jp.nextToken();
+                } catch (IOException e) {
+                    throw Throwables.propagate(e);
+                }
+                return true;
+            }
+
+            @Override
+            public T next() {
+                if (!hasNext()) {
+                    throw new NoSuchElementException();
+                }
+                T result = next;
+                next = null;
+                return result;
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException("remove");
+            }
+        };
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/RowCursor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/RowCursor.java
new file mode 100644
index 0000000000..450193cbb2
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/RowCursor.java
@@ -0,0 +1,22 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+public interface RowCursor extends Cursor<Object[]> {
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/RowCursorFactory.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/RowCursorFactory.java
new file mode 100644
index 0000000000..0186d6cf14
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/RowCursorFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.http.HttpClient;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import io.druid.data.input.Row;
+import io.druid.query.Query;
+import io.druid.query.groupby.GroupByQuery;
+import io.druid.query.scan.ScanQuery;
+import io.druid.query.scan.ScanResultValue;
+
+public class RowCursorFactory {
+    public static final TypeReference<Row> ROW_TYPE_REFERENCE = new TypeReference<Row>() {
+    };
+    public static final TypeReference<ScanResultValue> SCAN_TYPE_REFERENCE = new TypeReference<ScanResultValue>() {
+    };
+
+    private final HttpClient httpClient;
+    private final ObjectMapper objectMapper;
+
+    public RowCursorFactory(HttpClient httpClient, ObjectMapper objectMapper) {
+        this.httpClient = httpClient;
+        this.objectMapper = objectMapper;
+    }
+
+    public RowCursor createRowCursor(DruidSchema schema, Query<?> query, StorageContext context) {
+        String brokerHost = context.getCuboid().getCubeDesc().getConfig().getDruidBrokerHost();
+
+        if (query instanceof GroupByQuery) {
+            DruidClient<Row> client = new HttpDruidClient<>(httpClient, objectMapper, ROW_TYPE_REFERENCE, brokerHost);
+            return new GroupByRowCursor(schema, client, (GroupByQuery) query, context);
+        }
+
+        if (query instanceof ScanQuery) {
+            DruidClient<ScanResultValue> client = new HttpDruidClient<>(httpClient, objectMapper, SCAN_TYPE_REFERENCE, brokerHost);
+            return new ScanRowCursor(schema, client, (ScanQuery) query, context);
+        }
+
+        throw new IllegalArgumentException("Unknown query type " + query.getType());
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/ScanRowCursor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/ScanRowCursor.java
new file mode 100644
index 0000000000..f35aba952b
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/cursor/ScanRowCursor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.storage.druid.read.cursor;
+
+import java.util.List;
+
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.druid.DruidSchema;
+
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.scan.ScanQuery;
+import io.druid.query.scan.ScanResultValue;
+
+public class ScanRowCursor extends AbstractRowCursor<ScanResultValue> {
+    private final AggregatorFactory[] factories;
+
+    public ScanRowCursor(DruidSchema schema, DruidClient<ScanResultValue> client, ScanQuery query, StorageContext context) {
+        super(schema, client, query, context);
+        this.factories = schema.getAggregators();
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected boolean convert(ScanResultValue result, int index, Object[] row) {
+        List<List<Object>> rows = (List<List<Object>>) result.getEvents();
+        if (index >= rows.size()) {
+            return false;
+        }
+
+        List<Object> values = rows.get(index);
+        final int numDims = schema.getDimensions().size();
+        for (int i = 0; i < values.size(); i++) {
+            Object value = values.get(i);
+            if (i >= numDims) { // measures
+                value = factories[i - numDims].deserialize(value);
+            }
+            row[i] = convertComplexValue(value);
+        }
+        return true;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/FilterCondition.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/FilterCondition.java
new file mode 100644
index 0000000000..e8cf958c5d
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/FilterCondition.java
@@ -0,0 +1,55 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.List;
+
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.joda.time.Interval;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import io.druid.query.filter.DimFilter;
+
+public class FilterCondition {
+    public static final FilterCondition MATCH_EVERYTHING = FilterCondition.of(ImmutableList.of(DruidSchema.ETERNITY_INTERVAL), null);
+    public static final FilterCondition MATCH_NOTHING = FilterCondition.of(ImmutableList.<Interval> of(), null);
+
+    private final List<Interval> intervals;
+    private final DimFilter dimFilter;
+
+    private FilterCondition(List<Interval> intervals, DimFilter dimFilter) {
+        Preconditions.checkNotNull(intervals, "intervals is null");
+        this.intervals = intervals;
+        this.dimFilter = dimFilter;
+    }
+
+    public static FilterCondition of(List<Interval> intervals, DimFilter dimFilter) {
+        return new FilterCondition(intervals, dimFilter);
+    }
+
+    public List<Interval> getIntervals() {
+        return intervals;
+    }
+
+    public DimFilter getDimFilter() {
+        return dimFilter;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/FunctionConverters.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/FunctionConverters.java
new file mode 100644
index 0000000000..58ea1d729c
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/FunctionConverters.java
@@ -0,0 +1,121 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.kylin.metadata.filter.BuiltInFunctionTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+
+import com.google.common.collect.Lists;
+
+import io.druid.query.extraction.CascadeExtractionFn;
+import io.druid.query.extraction.ExtractionFn;
+import io.druid.query.extraction.LowerExtractionFn;
+import io.druid.query.extraction.StrlenExtractionFn;
+import io.druid.query.extraction.TimeFormatExtractionFn;
+import io.druid.query.extraction.UpperExtractionFn;
+
+public class FunctionConverters {
+    private static final ExtractionFn LOWER_EXTRACT = new LowerExtractionFn(null);
+    private static final ExtractionFn UPPER_EXTRACT = new UpperExtractionFn(null);
+
+    public static ExtractionFn convert(BuiltInFunctionTupleFilter filter) {
+        ExtractionFn childFn = null;
+        TupleFilter columnContainer = filter.getColumnContainerFilter();
+        if (columnContainer instanceof BuiltInFunctionTupleFilter) {
+            childFn = convert((BuiltInFunctionTupleFilter) columnContainer);
+            if (childFn == null) {
+                return null;
+            }
+        }
+
+        ExtractionFn parentFn = null;
+        switch (filter.getName()) {
+        case "EXTRACT_DATE": {
+            ConstantTupleFilter constantTupleFilter = filter.getConstantTupleFilter();
+            String timeUnit = (String) constantTupleFilter.getValues().iterator().next();
+            String format;
+            switch (timeUnit) {
+            case "YEAR":
+                format = "y";
+                break;
+            case "MONTH":
+                format = "M";
+                break;
+            case "DAY":
+                format = "d";
+                break;
+            default:
+                return null;
+            }
+            parentFn = new TimeFormatExtractionFn(format, null, null, null, true);
+            break;
+        }
+        case "UPPER":
+            parentFn = UPPER_EXTRACT;
+            break;
+        case "LOWER":
+            parentFn = LOWER_EXTRACT;
+            break;
+        case "CHARLENGTH":
+            parentFn = StrlenExtractionFn.instance();
+            break;
+        case "SUBSTRING": // TODO SubstringDimExtractionFn
+        default:
+            parentFn = null;
+        }
+
+        return compose(parentFn, childFn);
+    }
+
+    /**
+     * Compose f and g, returning an ExtractionFn that computes f(g(x)).
+     * @param f function
+     * @param g function
+     * @return composed function
+     */
+    private static ExtractionFn compose(final ExtractionFn f, final ExtractionFn g) {
+        if (g == null) {
+            return f;
+        }
+        if (f == null) {
+            return null;
+        }
+
+        final List<ExtractionFn> extractionFns = Lists.newArrayList();
+
+        // Apply g, then f, unwrapping if they are already cascades.
+        if (g instanceof CascadeExtractionFn) {
+            extractionFns.addAll(Arrays.asList(((CascadeExtractionFn) g).getExtractionFns()));
+        } else {
+            extractionFns.add(g);
+        }
+
+        if (f instanceof CascadeExtractionFn) {
+            extractionFns.addAll(Arrays.asList(((CascadeExtractionFn) f).getExtractionFns()));
+        } else {
+            extractionFns.add(f);
+        }
+
+        return new CascadeExtractionFn(extractionFns.toArray(new ExtractionFn[extractionFns.size()]));
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/MoreDimFilters.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/MoreDimFilters.java
new file mode 100644
index 0000000000..4e66171650
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/MoreDimFilters.java
@@ -0,0 +1,147 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+
+import io.druid.js.JavaScriptConfig;
+import io.druid.query.filter.AndDimFilter;
+import io.druid.query.filter.BoundDimFilter;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.InDimFilter;
+import io.druid.query.filter.JavaScriptDimFilter;
+import io.druid.query.filter.NotDimFilter;
+import io.druid.query.filter.OrDimFilter;
+import io.druid.query.filter.SelectorDimFilter;
+
+public class MoreDimFilters {
+    public static final DimFilter ALWAYS_FALSE = new JavaScriptDimFilter("dummy", "function(x){return false;}", null, new JavaScriptConfig(false));
+    public static final DimFilter ALWAYS_TRUE = new JavaScriptDimFilter("dummy", "function(x){return true;}", null, new JavaScriptConfig(false));
+
+    public static String getDimension(DimFilter filter) {
+        if (filter instanceof SelectorDimFilter) {
+            return ((SelectorDimFilter) filter).getDimension();
+        }
+        if (filter instanceof BoundDimFilter) {
+            return ((BoundDimFilter) filter).getDimension();
+        }
+        if (filter instanceof InDimFilter) {
+            return ((InDimFilter) filter).getDimension();
+        }
+        return null; // TODO support more filters
+    }
+
+    public static DimFilter and(List<DimFilter> children) {
+        Preconditions.checkNotNull(children);
+        Preconditions.checkArgument(!children.isEmpty());
+
+        List<DimFilter> newChildren = new ArrayList<>();
+        for (DimFilter child : children) {
+            if (child == ALWAYS_FALSE) {
+                return child; // and with false, short circuit
+            }
+            if (child == ALWAYS_TRUE) {
+                continue; // and with true, ignore
+            }
+            if (child instanceof AndDimFilter) {
+                newChildren.addAll(((AndDimFilter) child).getFields());
+            } else {
+                newChildren.add(child);
+            }
+        }
+
+        if (newChildren.isEmpty()) {
+            return ALWAYS_TRUE; // and with all true
+        }
+        if (newChildren.size() == 1) {
+            return newChildren.get(0);
+        }
+        return new AndDimFilter(newChildren);
+    }
+
+    public static DimFilter or(List<DimFilter> children) {
+        Preconditions.checkNotNull(children);
+        Preconditions.checkArgument(!children.isEmpty());
+
+        List<DimFilter> newChildren = new ArrayList<>();
+        for (DimFilter child : children) {
+            if (child == ALWAYS_TRUE) {
+                return child; // or with true, short circuit
+            }
+            if (child == ALWAYS_FALSE) {
+                continue; // or with false, ignore
+            }
+            if (child instanceof OrDimFilter) {
+                newChildren.addAll(((OrDimFilter) child).getFields());
+            } else {
+                newChildren.add(child);
+            }
+        }
+
+        if (newChildren.isEmpty()) {
+            return ALWAYS_FALSE; // or with all false
+        }
+        if (newChildren.size() == 1) {
+            return newChildren.get(0);
+        }
+        return new OrDimFilter(children);
+    }
+
+    public static DimFilter not(DimFilter child) {
+        Preconditions.checkNotNull(child);
+
+        if (child == ALWAYS_TRUE) {
+            return ALWAYS_FALSE;
+        }
+        if (child == ALWAYS_FALSE) {
+            return ALWAYS_TRUE;
+        }
+        if (child instanceof NotDimFilter) {
+            return ((NotDimFilter) child).getField();
+        }
+        return new NotDimFilter(child);
+    }
+
+    public static int leafCount(DimFilter filter) {
+        if (filter instanceof AndDimFilter) {
+            int count = 0;
+            for (DimFilter child : ((AndDimFilter) filter).getFields()) {
+                count += leafCount(child);
+            }
+            return count;
+        }
+
+        if (filter instanceof OrDimFilter) {
+            int count = 0;
+            for (DimFilter child : ((OrDimFilter) filter).getFields()) {
+                count += leafCount(child);
+            }
+            return count;
+        }
+
+        if (filter instanceof NotDimFilter) {
+            return leafCount(((NotDimFilter) filter).getField());
+        }
+
+        return 1;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/PruneIntervalsProcessor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/PruneIntervalsProcessor.java
new file mode 100644
index 0000000000..d19e9cde4e
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/PruneIntervalsProcessor.java
@@ -0,0 +1,260 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeSet;
+import io.druid.query.filter.AndDimFilter;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.NotDimFilter;
+import io.druid.query.filter.OrDimFilter;
+import io.druid.query.ordering.StringComparator;
+import org.apache.commons.lang3.time.FastDateFormat;
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.PartitionDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.NameMapping;
+import org.joda.time.Interval;
+
+import java.text.ParseException;
+import java.util.List;
+import java.util.Locale;
+
+public class PruneIntervalsProcessor implements Function<FilterCondition, FilterCondition> {
+    private final CubeInstance cube;
+
+    private Function<String, Long> formatter;
+    // not null only if formatter is not null
+    private String partCol;
+    private StringComparator partComparator;
+    private RangeKey partKey;
+
+    public PruneIntervalsProcessor(CubeInstance cube, NameMapping mapping) {
+        this.cube = cube;
+
+        PartitionDesc partitionDesc = cube.getDescriptor().getModel().getPartitionDesc();
+        final TblColRef partColRef = partitionDesc.getPartitionDateColumnRef();
+        boolean isDatePattern = DateFormat.isDatePattern(partitionDesc.getPartitionDateFormat());
+
+        if (partitionDesc.isPartitioned() && isDatePattern && partitionDesc.getPartitionTimeColumnRef() == null) {
+            final FastDateFormat format = DateFormat.getDateFormat(partitionDesc.getPartitionDateFormat());
+            DataType dataType = partColRef.getType();
+            if (dataType.isDate()) {
+                formatter = new Function<String, Long>() {
+                    @Override
+                    public Long apply(String input) {
+                        return Long.parseLong(input);
+                    }
+                };
+            } else if (dataType.isInt() || dataType.isBigInt() || dataType.isStringFamily()) {
+                formatter = new Function<String, Long>() {
+                    @Override
+                    public Long apply(String input) {
+                        try {
+                            return format.parse(input).getTime();
+                        } catch (ParseException e) {
+                            String err = String.format(Locale.ROOT,
+                                    "Failed to format value %s of partition column %s: %s",
+                                    input, partColRef, e.getMessage());
+                            throw new RuntimeException(err, e);
+                        }
+                    }
+                };
+            }
+        }
+
+        if (formatter != null) {
+            partCol = mapping.getDimFieldName(partColRef);
+            partComparator = DruidSchema.dimensionComparator(partColRef.getType());
+            partKey = new RangeKey(partCol, null, partComparator);
+        }
+    }
+
+    @Override
+    public FilterCondition apply(FilterCondition input) {
+        final DimFilter filter = input.getDimFilter();
+        if (filter == null || partCol == null) {
+            return input;
+        }
+
+        Result result = extractTimeRanges(filter);
+        if (result.isUnable()) {
+            return input;
+        }
+
+        // prune segments via extracted time ranges
+        RangeSet<Long> segmentRanges = TreeRangeSet.create();
+        for (CubeSegment segment : cube.getSegments(SegmentStatusEnum.READY)) {
+            Range<Long> segmentRange = Range.closedOpen(segment.getTSRange().start.v, segment.getTSRange().start.v);
+            boolean containQueryRange = !result.timeRanges.subRangeSet(segmentRange).isEmpty();
+            if (segment.getInputRecords() > 0 && containQueryRange) {
+                segmentRanges.add(segmentRange);
+            }
+        }
+
+        List<Interval> intervals = Lists.newArrayList();
+        for (Range<Long> range : segmentRanges.asRanges()) {
+            intervals.add(new Interval(range.lowerEndpoint(), range.upperEndpoint()));
+        }
+        return FilterCondition.of(intervals, filter);
+    }
+
+    private Result extractTimeRanges(DimFilter filter) {
+        if (filter instanceof AndDimFilter) {
+            boolean allCompleteConverted = true;
+            List<RangeSet<Long>> rangeSets = Lists.newArrayList();
+
+            for (DimFilter child : ((AndDimFilter) filter).getFields()) {
+                Result childResult = extractTimeRanges(child);
+                allCompleteConverted = allCompleteConverted && childResult.isComplete();
+                if (childResult.timeRanges != null) {
+                    rangeSets.add(childResult.timeRanges);
+                }
+            }
+
+            if (rangeSets.isEmpty()) {
+                return Result.unable();
+            }
+            RangeSet<Long> merged = RangeSets.intersect(rangeSets);
+            if (allCompleteConverted) {
+                return Result.complete(merged);
+            } else {
+                return Result.partial(merged);
+            }
+
+        } else if (filter instanceof OrDimFilter) {
+            boolean allCompleteConverted = true;
+            List<RangeSet<Long>> rangeSets = Lists.newArrayList();
+
+            for (DimFilter child : ((OrDimFilter) filter).getFields()) {
+                Result childResult = extractTimeRanges(child);
+                if (childResult.isUnable()) {
+                    return Result.unable();
+                }
+
+                allCompleteConverted = allCompleteConverted && childResult.isComplete();
+                rangeSets.add(childResult.timeRanges);
+            }
+
+            RangeSet<Long> merged = RangeSets.union(rangeSets);
+            if (allCompleteConverted) {
+                return Result.complete(merged);
+            } else {
+                return Result.partial(merged);
+            }
+
+        } else if (filter instanceof NotDimFilter) {
+            Result result = extractTimeRanges(((NotDimFilter) filter).getField());
+            if (result.isComplete()) {
+                return Result.complete(result.timeRanges.complement());
+            }
+            return Result.unable();
+
+        } else {
+            RangeKey key = RangeKey.from(filter, partComparator);
+            if (!partKey.equals(key)) {
+                return Result.unable();
+            }
+            RangeSet<RangeValue> rangeSet = RangeSets.from(filter, partComparator);
+            try {
+                RangeSet<Long> longRangeSet = toLong(rangeSet, formatter);
+                return Result.complete(longRangeSet);
+
+            } catch (RuntimeException e) {
+                // parsing filter value could fail for int/string type partition column,
+                // e.g, "where part_dt = 'abcd'" is a valid sql if part_dt is of string type.
+                // in that case, we rely on druid to do the filter work.
+                return Result.unable();
+            }
+        }
+    }
+
+    private static RangeSet<Long> toLong(RangeSet<RangeValue> rangeSet, Function<String, Long> formatter) {
+        RangeSet<Long> result = TreeRangeSet.create();
+        for (Range<RangeValue> range : rangeSet.asRanges()) {
+            Range<Long> longRange;
+            if (!range.hasLowerBound() && !range.hasUpperBound()) {
+                longRange = Range.all();
+            } else if (range.hasLowerBound() && !range.hasUpperBound()) {
+                Long lower = formatter.apply(range.lowerEndpoint().getValue());
+                longRange = Range.downTo(lower, range.lowerBoundType());
+            } else if (!range.hasLowerBound() && range.hasUpperBound()) {
+                Long upper = formatter.apply(range.upperEndpoint().getValue());
+                longRange = Range.upTo(upper, range.upperBoundType());
+            } else {
+                Long lower = formatter.apply(range.lowerEndpoint().getValue());
+                Long upper = formatter.apply(range.upperEndpoint().getValue());
+                longRange = Range.range(lower, range.lowerBoundType(), upper, range.upperBoundType());
+            }
+            result.add(longRange);
+        }
+        return result;
+    }
+
+    private static class Result {
+        enum ResultType {
+            COMPLETE, // when a filter consists only partition column, like dt between '2017-12-01' and '2017-12-10'
+            PARTIAL, // when a filter can be converted to partition column filter AND other filter, like dt='2017-12-01' and city='Beijing'
+            UNABLE // when a filter can't be converted to partition column filter AND other filter, like dt='2017-12-01' or city='Beijing'
+        }
+
+        RangeSet<Long> timeRanges;
+        ResultType type;
+
+        private Result(RangeSet<Long> timeRanges, ResultType type) {
+            this.timeRanges = timeRanges;
+            this.type = type;
+        }
+
+        static Result complete(RangeSet<Long> rangeSet) {
+            Preconditions.checkNotNull(rangeSet);
+            return new Result(rangeSet, ResultType.COMPLETE);
+        }
+
+        static Result partial(RangeSet<Long> rangeSet) {
+            Preconditions.checkNotNull(rangeSet);
+            return new Result(rangeSet, ResultType.PARTIAL);
+        }
+
+        static Result unable() {
+            return new Result(null, ResultType.UNABLE);
+        }
+
+        public boolean isComplete() {
+            return type == ResultType.COMPLETE;
+        }
+
+        public boolean isPartial() {
+            return type == ResultType.PARTIAL;
+        }
+
+        public boolean isUnable() {
+            return type == ResultType.UNABLE;
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeKey.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeKey.java
new file mode 100644
index 0000000000..297c48ee60
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeKey.java
@@ -0,0 +1,88 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.Objects;
+
+import io.druid.query.extraction.ExtractionFn;
+import io.druid.query.filter.BoundDimFilter;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.InDimFilter;
+import io.druid.query.filter.SelectorDimFilter;
+import io.druid.query.ordering.StringComparator;
+
+public class RangeKey {
+    private final String dimension;
+    private final ExtractionFn extractionFn;
+    private final StringComparator comparator;
+
+    public RangeKey(String dimension, ExtractionFn extractionFn, StringComparator comparator) {
+        this.dimension = dimension;
+        this.extractionFn = extractionFn;
+        this.comparator = comparator;
+    }
+
+    public static RangeKey from(DimFilter filter, StringComparator comparator) {
+        if (filter instanceof SelectorDimFilter) {
+            SelectorDimFilter select = (SelectorDimFilter) filter;
+            return new RangeKey(select.getDimension(), select.getExtractionFn(), comparator);
+        }
+        if (filter instanceof BoundDimFilter) {
+            BoundDimFilter bound = (BoundDimFilter) filter;
+            return new RangeKey(bound.getDimension(), bound.getExtractionFn(), comparator);
+        }
+        if (filter instanceof InDimFilter) {
+            InDimFilter in = (InDimFilter) filter;
+            return new RangeKey(in.getDimension(), in.getExtractionFn(), comparator);
+        }
+        return null;
+    }
+
+    public String getDimension() {
+        return dimension;
+    }
+
+    public ExtractionFn getExtractionFn() {
+        return extractionFn;
+    }
+
+    public StringComparator getComparator() {
+        return comparator;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        RangeKey rangeKey = (RangeKey) o;
+        return Objects.equals(dimension, rangeKey.dimension) && Objects.equals(extractionFn, rangeKey.extractionFn) && Objects.equals(comparator, rangeKey.comparator);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(dimension, extractionFn, comparator);
+    }
+
+    @Override
+    public String toString() {
+        return "RangeKey{" + "dimension='" + dimension + '\'' + ", extractionFn=" + extractionFn + ", comparator=" + comparator + '}';
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeSets.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeSets.java
new file mode 100644
index 0000000000..242ae7d6be
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeSets.java
@@ -0,0 +1,145 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeSet;
+
+import io.druid.query.filter.BoundDimFilter;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.InDimFilter;
+import io.druid.query.filter.SelectorDimFilter;
+import io.druid.query.ordering.StringComparator;
+
+import java.util.List;
+
+public class RangeSets {
+
+    public static RangeSet<RangeValue> from(DimFilter filter, StringComparator comparator) {
+        if (filter instanceof SelectorDimFilter) {
+            SelectorDimFilter select = (SelectorDimFilter) filter;
+            RangeValue value = new RangeValue(select.getValue(), comparator);
+            return of(Range.singleton(value));
+        }
+        if (filter instanceof BoundDimFilter) {
+            BoundDimFilter bound = (BoundDimFilter) filter;
+            BoundType lowerType = bound.isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED;
+            BoundType upperType = bound.isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED;
+            RangeValue lowerValue = bound.hasLowerBound() ? new RangeValue(bound.getLower(), comparator) : null;
+            RangeValue upperValue = bound.hasUpperBound() ? new RangeValue(bound.getUpper(), comparator) : null;
+
+            Range range = null;
+            if (bound.hasLowerBound() && bound.hasUpperBound()) {
+                range = Range.range(lowerValue, lowerType, upperValue, upperType);
+            } else if (bound.hasLowerBound()) {
+                range = Range.downTo(lowerValue, lowerType);
+            } else if (bound.hasUpperBound()) {
+                range = Range.upTo(upperValue, upperType);
+            }
+            return of(range);
+        }
+        if (filter instanceof InDimFilter) {
+            InDimFilter in = (InDimFilter) filter;
+            RangeSet<RangeValue> result = TreeRangeSet.create();
+            for (String value : in.getValues()) {
+                result.add(Range.singleton(new RangeValue(value, comparator)));
+            }
+            return result;
+        }
+        return null;
+    }
+
+    public static BoundDimFilter rangeToFilter(RangeKey key, Range<RangeValue> range) {
+        return new BoundDimFilter(
+                key.getDimension(),
+                range.hasLowerBound() ? range.lowerEndpoint().getValue() : null,
+                range.hasUpperBound() ? range.upperEndpoint().getValue() : null,
+                range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN,
+                range.hasUpperBound() && range.upperBoundType() == BoundType.OPEN,
+                null,
+                key.getExtractionFn(),
+                key.getComparator()
+        );
+    }
+
+    public static DimFilter rangeSetsToFilter(RangeKey key, RangeSet<RangeValue> rangeSet) {
+        if (rangeSet.isEmpty()) {
+            return MoreDimFilters.ALWAYS_FALSE;
+        }
+
+        List<String> values = Lists.newArrayList();
+        List<DimFilter> filters = Lists.newArrayList();
+
+        for (Range<RangeValue> range : rangeSet.asRanges()) {
+            if (!range.hasLowerBound() && !range.hasUpperBound()) {
+                return MoreDimFilters.ALWAYS_TRUE;
+            }
+            if (range.hasLowerBound() && range.hasUpperBound() && range.lowerEndpoint().equals(range.upperEndpoint())) {
+                values.add(range.lowerEndpoint().getValue()); // range match single value
+            } else {
+                filters.add(rangeToFilter(key, range));
+            }
+        }
+
+        if (!values.isEmpty()) {
+            filters.add(new InDimFilter(key.getDimension(), values, key.getExtractionFn()));
+        }
+
+        return MoreDimFilters.or(filters);
+    }
+
+    public static <T extends Comparable<T>> RangeSet<T> of(Range<T> range) {
+        RangeSet<T> set = TreeRangeSet.create();
+        set.add(range);
+        return set;
+    }
+
+    /**
+     * Intersects a set of rangeSets, or returns empty set if the set is empty.
+     */
+    public static <T extends Comparable<T>> RangeSet<T> intersect(final Iterable<RangeSet<T>> rangeSets) {
+        RangeSet<T> rangeSet = null;
+        for (final RangeSet<T> set : rangeSets) {
+            if (rangeSet == null) {
+                rangeSet = TreeRangeSet.create();
+                rangeSet.addAll(set);
+            } else {
+                rangeSet.removeAll(set.complement());
+            }
+        }
+        if (rangeSet == null) {
+            return TreeRangeSet.create();
+        }
+        return rangeSet;
+    }
+
+    /**
+     * Unions a set of rangeSets, or returns empty set if the set is empty.
+     */
+    public static <T extends Comparable<T>> RangeSet<T> union(final Iterable<RangeSet<T>> rangeSets) {
+        final RangeSet<T> rangeSet = TreeRangeSet.create();
+        for (RangeSet<T> set : rangeSets) {
+            rangeSet.addAll(set);
+        }
+        return rangeSet;
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeValue.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeValue.java
new file mode 100644
index 0000000000..6e529bdd23
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RangeValue.java
@@ -0,0 +1,66 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.Objects;
+
+import io.druid.query.ordering.StringComparator;
+
+public class RangeValue implements Comparable<RangeValue> {
+    private final String value;
+    private final StringComparator comparator;
+
+    public RangeValue(String value, StringComparator comparator) {
+        this.value = value;
+        this.comparator = comparator;
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public StringComparator getComparator() {
+        return comparator;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        RangeValue rangeValue = (RangeValue) o;
+        return Objects.equals(value, rangeValue.value) && Objects.equals(comparator, rangeValue.comparator);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(value, comparator);
+    }
+
+    @Override
+    public String toString() {
+        return value;
+    }
+
+    @Override
+    public int compareTo(RangeValue o) {
+        return comparator.compare(value, o.value);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RemoveMarkerProcessor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RemoveMarkerProcessor.java
new file mode 100644
index 0000000000..723090e999
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/RemoveMarkerProcessor.java
@@ -0,0 +1,60 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import com.google.common.base.Function;
+import io.druid.query.filter.AndDimFilter;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.NotDimFilter;
+import io.druid.query.filter.OrDimFilter;
+
+public class RemoveMarkerProcessor implements Function<FilterCondition, FilterCondition> {
+    public static final RemoveMarkerProcessor SINGLETON = new RemoveMarkerProcessor();
+
+    private RemoveMarkerProcessor() {}
+
+    @Override
+    public FilterCondition apply(FilterCondition condition) {
+        DimFilter filter = condition.getDimFilter();
+        if (filter == MoreDimFilters.ALWAYS_TRUE) {
+            return FilterCondition.MATCH_EVERYTHING;
+        }
+        if (filter == MoreDimFilters.ALWAYS_FALSE) {
+            return FilterCondition.MATCH_NOTHING;
+        }
+        validateNoMarkerFilter(filter);
+        return condition;
+    }
+
+    private void validateNoMarkerFilter(DimFilter filter) {
+        if (filter instanceof AndDimFilter) {
+            for (DimFilter child : ((AndDimFilter) filter).getFields()) {
+                validateNoMarkerFilter(child);
+            }
+        } else if (filter instanceof OrDimFilter) {
+            for (DimFilter child : ((OrDimFilter) filter).getFields()) {
+                validateNoMarkerFilter(child);
+            }
+        } else if (filter instanceof NotDimFilter) {
+            validateNoMarkerFilter(((NotDimFilter) filter).getField());
+        } else if (filter == MoreDimFilters.ALWAYS_TRUE || filter == MoreDimFilters.ALWAYS_FALSE) {
+            throw new IllegalStateException("still have mark filter");
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/ToDimFilterVisitor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/ToDimFilterVisitor.java
new file mode 100644
index 0000000000..c62b3ffdcc
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/ToDimFilterVisitor.java
@@ -0,0 +1,234 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.RangeSet;
+import org.apache.kylin.metadata.filter.BuiltInFunctionTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilterVisitor2;
+import org.apache.kylin.metadata.filter.TupleFilterVisitor2Adaptor;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.NameMapping;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import io.druid.query.extraction.ExtractionFn;
+import io.druid.query.filter.BoundDimFilter;
+import io.druid.query.filter.DimFilter;
+import io.druid.query.filter.InDimFilter;
+import io.druid.query.filter.LikeDimFilter;
+import io.druid.query.filter.NotDimFilter;
+import io.druid.query.filter.SelectorDimFilter;
+import io.druid.query.ordering.StringComparator;
+
+public class ToDimFilterVisitor implements TupleFilterVisitor2<DimFilter> {
+
+    private final NameMapping mapping;
+    private final List<TblColRef> dimensions;
+
+    private Map<String, StringComparator> comparators = Maps.newHashMap();
+
+    public ToDimFilterVisitor(NameMapping mapping, List<TblColRef> dimensions) {
+        this.mapping = mapping;
+        this.dimensions = dimensions;
+    }
+
+    private StringComparator getComparator(DimFilter filter) {
+        String dimension = MoreDimFilters.getDimension(filter);
+        return comparators.get(dimension);
+    }
+
+    private DimFilter makeCompareFilter(TblColRef column, TupleFilter.FilterOperatorEnum op, Set<?> values, Object firstValue, ExtractionFn extractionFn) {
+        final String dimension = mapping.getDimFieldName(column);
+        StringComparator comparator = comparators.get(dimension);
+        if (comparator == null) {
+            comparator = DruidSchema.dimensionComparator(column.getType());
+            comparators.put(dimension, comparator);
+        }
+
+        // FIXME toString only works for string, float, double, int, long?
+        String stringValue = null;
+        if (firstValue != null) {
+            stringValue = firstValue.toString();
+        }
+        final Set<String> stringSet = new HashSet<>();
+        for (Object value : values) {
+            stringSet.add(value.toString());
+        }
+
+        switch (op) {
+        case EQ:
+            return new SelectorDimFilter(dimension, stringValue, extractionFn);
+        case NEQ:
+            return new NotDimFilter(new SelectorDimFilter(dimension, stringValue, extractionFn));
+        case LT:
+            return new BoundDimFilter(dimension, null, stringValue, null, true, null, extractionFn, comparator);
+        case LTE:
+            return new BoundDimFilter(dimension, null, stringValue, null, false, null, extractionFn, comparator);
+        case GT:
+            return new BoundDimFilter(dimension, stringValue, null, true, null, null, extractionFn, comparator);
+        case GTE:
+            return new BoundDimFilter(dimension, stringValue, null, false, null, null, extractionFn, comparator);
+        case ISNULL:
+            return new SelectorDimFilter(dimension, null, null);
+        case ISNOTNULL:
+            return new NotDimFilter(new SelectorDimFilter(dimension, null, null));
+        case IN:
+            return new InDimFilter(dimension, stringSet, extractionFn);
+        case NOTIN:
+            return new NotDimFilter(new InDimFilter(dimension, stringSet, extractionFn));
+        default:
+            throw new AssertionError("Illegal op for CompareTupleFilter: " + op);
+        }
+    }
+
+    @Override
+    public DimFilter visitColumnCompare(CompareTupleFilter originFilter, TblColRef column, TupleFilter.FilterOperatorEnum op, Set<?> values, Object firstValue) {
+        assert dimensions.contains(column);
+        assert !values.isEmpty() || op == TupleFilter.FilterOperatorEnum.ISNOTNULL ||  op == TupleFilter.FilterOperatorEnum.ISNULL;
+
+        return makeCompareFilter(column, op, values, firstValue, null);
+    }
+
+    @Override
+    public DimFilter visitColumnLike(BuiltInFunctionTupleFilter originFilter, TblColRef column, String pattern, boolean reversed) {
+        assert dimensions.contains(column);
+
+        ExtractionFn extractionFn = null;
+        if (originFilter.getColumnContainerFilter() instanceof BuiltInFunctionTupleFilter) {
+            // nested case: lower(c) like 'x'
+            extractionFn = FunctionConverters.convert((BuiltInFunctionTupleFilter) originFilter.getColumnContainerFilter());
+            if (extractionFn == null) {
+                return visitUnsupported(originFilter);
+            }
+        }
+
+        String dimension = mapping.getDimFieldName(column);
+        LikeDimFilter likeDimFilter = new LikeDimFilter(dimension, pattern, null, extractionFn);
+        return reversed ? new NotDimFilter(likeDimFilter) : likeDimFilter;
+    }
+
+    @Override
+    public DimFilter visitColumnFunction(CompareTupleFilter originFilter, BuiltInFunctionTupleFilter function, TupleFilter.FilterOperatorEnum op, Set<?> values, Object firstValue) {
+        TblColRef column = function.getColumn();
+        assert dimensions.contains(column);
+
+        ExtractionFn extractionFn = FunctionConverters.convert(function);
+        if (extractionFn == null) {
+            return visitUnsupported(originFilter);
+        }
+
+        return makeCompareFilter(column, op, values, firstValue, extractionFn);
+    }
+
+    @Override
+    public DimFilter visitAnd(LogicalTupleFilter originFilter, List<? extends TupleFilter> children, TupleFilterVisitor2Adaptor<DimFilter> adaptor) {
+        List<DimFilter> childFilters = Lists.newArrayListWithCapacity(children.size());
+        for (TupleFilter child : children) {
+            childFilters.add(child.accept(adaptor));
+        }
+        return simplify(childFilters, false);
+    }
+
+    @Override
+    public DimFilter visitOr(LogicalTupleFilter originFilter, List<? extends TupleFilter> children, TupleFilterVisitor2Adaptor<DimFilter> adaptor) {
+        List<DimFilter> childFilters = Lists.newArrayListWithCapacity(children.size());
+        for (TupleFilter child : children) {
+            childFilters.add(child.accept(adaptor));
+        }
+        return simplify(childFilters, true);
+    }
+
+    private DimFilter simplify(List<DimFilter> children, boolean disjunction) {
+        List<DimFilter> newChildren = Lists.newArrayList(children);
+
+        ListMultimap<RangeKey, DimFilter> columnToFilter = ArrayListMultimap.create();
+        for (DimFilter child : children) {
+            RangeKey key = RangeKey.from(child, getComparator(child));
+            if (key != null) {
+                columnToFilter.put(key, child);
+            }
+        }
+
+        // try to simplify filters on each column
+        for (RangeKey key : columnToFilter.keySet()) {
+            List<DimFilter> filters = columnToFilter.get(key);
+            if (filters.size() == 1) {
+                continue; // can't be simplified any more
+            }
+            // determine value range
+            List<RangeSet<RangeValue>> rangeSets = Lists.newArrayList();
+            for (DimFilter filter : filters) {
+                rangeSets.add(RangeSets.from(filter, getComparator(filter)));
+            }
+            RangeSet<RangeValue> finalRangeSet = disjunction ? RangeSets.union(rangeSets) : RangeSets.intersect(rangeSets);
+
+            // convert range set back to filter
+            DimFilter newFilter = RangeSets.rangeSetsToFilter(key, finalRangeSet);
+            if (newFilter == MoreDimFilters.ALWAYS_TRUE && disjunction) {
+                return newFilter; // or short-circuit
+            }
+            if (newFilter == MoreDimFilters.ALWAYS_FALSE && !disjunction) {
+                return newFilter; // and short-circuit
+            }
+            int leafCount = MoreDimFilters.leafCount(newFilter);
+            if (leafCount < filters.size()) { // found simplification
+                newChildren.removeAll(filters);
+                newChildren.add(newFilter);
+            }
+        }
+
+        return disjunction ? MoreDimFilters.or(newChildren) : MoreDimFilters.and(newChildren);
+    }
+
+    @Override
+    public DimFilter visitNot(LogicalTupleFilter originFilter, TupleFilter child, TupleFilterVisitor2Adaptor<DimFilter> adaptor) {
+        DimFilter childFilter = child.accept(adaptor);
+        return MoreDimFilters.not(childFilter);
+    }
+
+    @Override
+    public DimFilter visitConstant(ConstantTupleFilter originFilter) {
+        // only happens on root filter
+        if (originFilter == ConstantTupleFilter.TRUE) {
+            return MoreDimFilters.ALWAYS_TRUE;
+        }
+        if (originFilter == ConstantTupleFilter.FALSE) {
+            return MoreDimFilters.ALWAYS_FALSE;
+        }
+        throw new AssertionError("other constant filter");
+    }
+
+    @Override
+    public DimFilter visitUnsupported(TupleFilter originFilter) {
+        throw new UnsupportedFilterException(originFilter.toString());
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/TranslateDerivedVisitor.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/TranslateDerivedVisitor.java
new file mode 100644
index 0000000000..3e85b9ff90
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/TranslateDerivedVisitor.java
@@ -0,0 +1,299 @@
+/*
+ * 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.storage.druid.read.filter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.kylin.common.util.TimeUtil;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
+import org.apache.kylin.dict.lookup.LookupStringTable;
+import org.apache.kylin.metadata.filter.BuiltInFunctionTupleFilter;
+import org.apache.kylin.metadata.filter.ColumnTupleFilter;
+import org.apache.kylin.metadata.filter.CompareTupleFilter;
+import org.apache.kylin.metadata.filter.ConstantTupleFilter;
+import org.apache.kylin.metadata.filter.FilterCodeSystemFactory;
+import org.apache.kylin.metadata.filter.IFilterCodeSystem;
+import org.apache.kylin.metadata.filter.LogicalTupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.filter.TupleFilterVisitor2;
+import org.apache.kylin.metadata.filter.TupleFilterVisitor2Adaptor;
+import org.apache.kylin.metadata.filter.TupleFilters;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
+
+import com.google.common.collect.Lists;
+import org.apache.kylin.storage.druid.read.LookupTableCache;
+
+/**
+ * Bottom up traverse filter tree, translate derived column filter to host column filter.
+ */
+public class TranslateDerivedVisitor implements TupleFilterVisitor2<TupleFilter> {
+    private final CubeDesc desc;
+    private final LookupTableCache lookupCache;
+
+    public TranslateDerivedVisitor(CubeDesc desc, LookupTableCache lookupCache) {
+        this.desc = desc;
+        this.lookupCache = lookupCache;
+    }
+
+    private void checkNotFilteringOnExtendedColumn(TblColRef column) {
+        if (desc.isExtendedColumn(column)) {
+            throw new CubeDesc.CannotFilterExtendedColumnException(column);
+        }
+    }
+
+    private List<String[]> collectAllMatchingHostRecords(TblColRef derivedColumn, DeriveInfo hostInfo, DerivedColumnMatcher matcher) {
+        int derivedIndex = derivedColumn.getColumnDesc().getZeroBasedIndex();
+
+        TblColRef[] pkCols = hostInfo.join.getPrimaryKeyColumns();
+        int[] pkIndices = new int[pkCols.length];
+        for (int i = 0; i < pkCols.length; i++) {
+            pkIndices[i] = pkCols[i].getColumnDesc().getZeroBasedIndex();
+        }
+
+        List<String[]> result = new ArrayList<>();
+        //FixMe currently we only support LookupStringTable
+        LookupStringTable lookup = (LookupStringTable) lookupCache.get(hostInfo.join);
+
+        for (String[] row : lookup.getAllRows()) {
+            if (matcher.match(row[derivedIndex])) {
+                String[] hostValues = new String[pkIndices.length];
+                for (int i = 0; i < pkIndices.length; i++) {
+                    hostValues[i] = row[pkIndices[i]];
+                }
+                result.add(hostValues);
+            }
+        }
+        return result;
+    }
+
+    private TupleFilter newHostColumnFilter(TblColRef[] hostColumns, List<String[]> hostRecords) {
+        if (hostRecords.size() > desc.getConfig().getDruidDerivedInThreshold()) {
+            throw new UnsupportedFilterException("too many rows matching derived column filter");
+        }
+
+        if (hostRecords.isEmpty()) {
+            return ConstantTupleFilter.FALSE;
+        }
+
+        if (hostColumns.length == 1) {
+            List<String> values = Lists.newArrayListWithExpectedSize(hostRecords.size());
+            for (String[] records : hostRecords) {
+                values.add(records[0]);
+            }
+            return TupleFilters.compare(hostColumns[0], FilterOperatorEnum.IN, values);
+        }
+
+        List<TupleFilter> ands = new ArrayList<>();
+        for (String[] hostValues : hostRecords) {
+            List<TupleFilter> equals = Lists.newArrayListWithCapacity(hostColumns.length);
+            for (int i = 0; i < hostColumns.length; i++) {
+                equals.add(TupleFilters.eq(hostColumns[i], hostValues[i]));
+            }
+            ands.add(TupleFilters.and(equals));
+        }
+        return TupleFilters.or(ands);
+    }
+
+    @Override
+    public TupleFilter visitColumnCompare(final CompareTupleFilter originFilter, final TblColRef column, FilterOperatorEnum op, Set<?> values, Object firstValue) {
+        checkNotFilteringOnExtendedColumn(column);
+        if (!desc.isDerived(column)) {
+            return originFilter;
+        }
+
+        DeriveInfo hostInfo = desc.getHostInfo(column);
+        TblColRef[] hostCols = hostInfo.columns;
+
+        if (hostInfo.type == CubeDesc.DeriveType.PK_FK) {
+            return TupleFilters.compare(hostCols[0], op, values);
+        }
+
+        // collect rows that matches filter
+        List<String[]> hostRecords = collectAllMatchingHostRecords(column, hostInfo, new DerivedColumnMatcher() {
+            SingleColumnTuple tuple = new SingleColumnTuple(column);
+            IFilterCodeSystem codeSystem = FilterCodeSystemFactory.getFilterCodeSystem(column.getColumnDesc().getType());
+
+            @Override
+            public boolean match(String value) {
+                tuple.value = value;
+                return originFilter.evaluate(tuple, codeSystem);
+            }
+        });
+
+        // translate to host column filter
+        return newHostColumnFilter(hostCols, hostRecords);
+    }
+
+    @Override
+    public TupleFilter visitColumnLike(final BuiltInFunctionTupleFilter originFilter, TblColRef column, final String pattern, final boolean reversed) {
+        checkNotFilteringOnExtendedColumn(column);
+        if (!desc.isDerived(column)) {
+            return originFilter;
+        }
+
+        DeriveInfo hostInfo = desc.getHostInfo(column);
+        TblColRef[] hostCols = hostInfo.columns;
+
+        if (hostInfo.type == CubeDesc.DeriveType.PK_FK && originFilter.getColumnContainerFilter() instanceof ColumnTupleFilter) {
+            return TupleFilters.like(hostCols[0], pattern, reversed);
+        }
+
+        final BuiltInFunctionTupleFilter newFilter = new BuiltInFunctionTupleFilter(originFilter.getName());
+        newFilter.addChild(new ColumnTupleFilter(column));
+        newFilter.addChild(originFilter.getConstantTupleFilter());
+        newFilter.setReversed(reversed);
+
+        List<String[]> hostRecords = collectAllMatchingHostRecords(column, hostInfo, new DerivedColumnMatcher() {
+            @Override
+            public boolean match(String value) {
+                try {
+                    if (originFilter.getColumnContainerFilter() instanceof BuiltInFunctionTupleFilter) {
+                        // lower(derived) like 'x'
+                        BuiltInFunctionTupleFilter columnFunction = (BuiltInFunctionTupleFilter) originFilter.getColumnContainerFilter();
+                        value = Objects.toString(invokeFunction(columnFunction, value));
+                    }
+                    boolean matched = (Boolean) newFilter.invokeFunction(value);
+                    return reversed ? !matched : matched;
+
+                } catch (Exception e) {
+                    throw new UnsupportedFilterException("Failed to evaluate LIKE expression: " + e.getMessage());
+                }
+            }
+        });
+
+        return newHostColumnFilter(hostCols, hostRecords);
+    }
+
+    @Override
+    public TupleFilter visitColumnFunction(CompareTupleFilter originFilter, final BuiltInFunctionTupleFilter function, FilterOperatorEnum op, Set<?> values, Object firstValue) {
+        final TblColRef column = function.getColumn();
+        checkNotFilteringOnExtendedColumn(column);
+        if (!desc.isDerived(column)) {
+            return originFilter;
+        }
+
+        DeriveInfo hostInfo = desc.getHostInfo(column);
+        TblColRef[] hostCols = hostInfo.columns;
+
+        final CompareTupleFilter newFilter = new CompareTupleFilter(op);
+        newFilter.addChild(new ColumnTupleFilter(column));
+        newFilter.addChild(new ConstantTupleFilter(values));
+
+        // collect rows that matches filter
+        List<String[]> hostRecords = collectAllMatchingHostRecords(column, hostInfo, new DerivedColumnMatcher() {
+            SingleColumnTuple tuple = new SingleColumnTuple(column);
+            IFilterCodeSystem codeSystem = FilterCodeSystemFactory.getFilterCodeSystem(column.getColumnDesc().getType());
+
+            @Override
+            public boolean match(String value) {
+                tuple.value = Objects.toString(invokeFunction(function, value));
+                return newFilter.evaluate(tuple, codeSystem);
+            }
+        });
+
+        // translate to host column filter
+        return newHostColumnFilter(hostCols, hostRecords);
+    }
+
+    private Object invokeFunction(BuiltInFunctionTupleFilter function, String value) {
+        // special case for extract(timeUnit from derived)
+        if ("EXTRACT_DATE".equals(function.getName())) {
+            if (value == null) {
+                return null;
+            }
+            ConstantTupleFilter constantTupleFilter = function.getConstantTupleFilter();
+            String timeUnit = (String) constantTupleFilter.getValues().iterator().next();
+            return TimeUtil.extract(timeUnit, value);
+        }
+
+        if (!function.isValid()) {
+            throw new UnsupportedOperationException("Function '" + function.getName() + "' is not supported");
+        }
+
+        try {
+            return function.invokeFunction(value);
+        } catch (Exception e) {
+            throw new UnsupportedFilterException("Function '" + function.getName() + "' failed to execute: " + e.getMessage());
+        }
+    }
+
+    @Override
+    public TupleFilter visitAnd(LogicalTupleFilter originFilter, List<? extends TupleFilter> children, TupleFilterVisitor2Adaptor<TupleFilter> adaptor) {
+        List<TupleFilter> newChildren = Lists.newArrayList();
+        for (TupleFilter child : children) {
+            TupleFilter newChild = child.accept(adaptor);
+            newChildren.add(newChild);
+        }
+        return TupleFilters.and(newChildren);
+    }
+
+    @Override
+    public TupleFilter visitOr(LogicalTupleFilter originFilter, List<? extends TupleFilter> children, TupleFilterVisitor2Adaptor<TupleFilter> adaptor) {
+        List<TupleFilter> newChildren = Lists.newArrayList();
+        for (TupleFilter child : children) {
+            TupleFilter newChild = child.accept(adaptor);
+            newChildren.add(newChild);
+        }
+        return TupleFilters.or(newChildren);
+    }
+
+    @Override
+    public TupleFilter visitNot(LogicalTupleFilter originFilter, TupleFilter child, TupleFilterVisitor2Adaptor<TupleFilter> adaptor) {
+        TupleFilter newChild = child.accept(adaptor);
+        return TupleFilters.not(newChild);
+    }
+
+    @Override
+    public TupleFilter visitConstant(ConstantTupleFilter originFilter) {
+        return originFilter;
+    }
+
+    @Override
+    public TupleFilter visitUnsupported(TupleFilter originFilter) {
+        throw new UnsupportedFilterException(originFilter.toString());
+    }
+
+    interface DerivedColumnMatcher {
+        boolean match(String value);
+    }
+
+    private static class SingleColumnTuple implements IEvaluatableTuple {
+        TblColRef col;
+        String value;
+
+        SingleColumnTuple(TblColRef col) {
+            this.col = col;
+        }
+
+        @Override
+        public Object getValue(TblColRef col) {
+            if (this.col.equals(col))
+                return value;
+            else
+                throw new IllegalArgumentException("unexpected column " + col);
+        }
+
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/UnsupportedFilterException.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/UnsupportedFilterException.java
new file mode 100644
index 0000000000..b6a537f9b1
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/read/filter/UnsupportedFilterException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.storage.druid.read.filter;
+
+public class UnsupportedFilterException extends RuntimeException {
+
+    public UnsupportedFilterException(String message) {
+        super("Can't push down filter to storage: " + message);
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/AnnounceDruidSegment.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/AnnounceDruidSegment.java
new file mode 100644
index 0000000000..86fe619f90
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/AnnounceDruidSegment.java
@@ -0,0 +1,116 @@
+/*
+ * 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.storage.druid.write;
+
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Set;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.storage.druid.common.MySQLConnector;
+import org.apache.kylin.storage.druid.common.SegmentPublishResult;
+import org.joda.time.DateTime;
+import org.skife.jdbi.v2.Handle;
+import org.skife.jdbi.v2.TransactionCallback;
+import org.skife.jdbi.v2.TransactionStatus;
+import org.skife.jdbi.v2.util.StringMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+import io.druid.jackson.DefaultObjectMapper;
+import io.druid.timeline.DataSegment;
+import io.druid.timeline.partition.NoneShardSpec;
+
+public class AnnounceDruidSegment {
+    protected static final Logger logger = LoggerFactory.getLogger(AnnounceDruidSegment.class);
+
+    private static final String QUOTE_STRING = "`";
+    private static final int DEFAULT_MAX_TRIES = 10;
+    private static final String segment_table = KylinConfig.getInstanceFromEnv().getDruidMysqlSegTabel();
+
+    private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
+
+    private final MySQLConnector connector;
+
+    public AnnounceDruidSegment() {
+        connector = new MySQLConnector();
+    }
+
+    public SegmentPublishResult announceHistoricalSegments(final Set<DataSegment> segments) throws IOException {
+        return connector.retryTransaction(new TransactionCallback<SegmentPublishResult>() {
+            @Override
+            public SegmentPublishResult inTransaction(final Handle handle, final TransactionStatus transactionStatus)
+                    throws Exception {
+                final Set<DataSegment> inserted = Sets.newHashSet();
+
+                for (final DataSegment segment : segments) {
+                    if (announceHistoricalSegment(handle, segment)) {
+                        inserted.add(segment);
+                    }
+                }
+
+                return new SegmentPublishResult(ImmutableSet.copyOf(inserted), true);
+            }
+        }, 3, DEFAULT_MAX_TRIES);
+    }
+
+    /**
+     * Attempts to insert a single segment to the database. If the segment already exists, will do nothing; although,
+     * this checking is imperfect and callers must be prepared to retry their entire transaction on exceptions.
+     *
+     * @return true if the segment was added, false if it already existed
+     */
+    private boolean announceHistoricalSegment(final Handle handle, final DataSegment segment) throws IOException {
+        try {
+            if (segmentExists(handle, segment)) {
+                logger.info("Found {} in DB, not updating DB", segment.getIdentifier());
+                return false;
+            }
+
+            // SELECT -> INSERT can fail due to races; callers must be prepared to retry.
+            // Avoiding ON DUPLICATE KEY since it's not portable.
+            // Avoiding try/catch since it may cause inadvertent transaction-splitting.
+            handle.createStatement(String.format(Locale.ROOT,
+                    "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) "
+                            + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
+                    segment_table, QUOTE_STRING)).bind("id", segment.getIdentifier())
+                    .bind("dataSource", segment.getDataSource()).bind("created_date", new DateTime().toString())
+                    .bind("start", segment.getInterval().getStart().toString())
+                    .bind("end", segment.getInterval().getEnd().toString())
+                    .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec))
+                    .bind("version", segment.getVersion()).bind("used", true)
+                    .bind("payload", jsonMapper.writeValueAsBytes(segment)).execute();
+            logger.info("Published segment {} to DB", segment.getIdentifier());
+        } catch (Exception e) {
+            logger.error("Exception inserting segment {} into DB", segment.getIdentifier(), e);
+            throw e;
+        }
+
+        return true;
+    }
+
+    private boolean segmentExists(final Handle handle, final DataSegment segment) {
+        return !handle.createQuery(String.format(Locale.ROOT, "SELECT id FROM %s WHERE id = :identifier", segment_table))
+                .bind("identifier", segment.getIdentifier()).map(StringMapper.FIRST).list().isEmpty();
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/CalculateShardsStep.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/CalculateShardsStep.java
new file mode 100644
index 0000000000..9c2fc17d32
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/CalculateShardsStep.java
@@ -0,0 +1,114 @@
+/*
+ * 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.storage.druid.write;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BufferedLogger;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.common.CubeStatsReader;
+import org.apache.kylin.engine.mr.common.CuboidShardUtil;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+// calculate and save cuboid->shards map based on cuboid size stats
+public class CalculateShardsStep extends AbstractExecutable {
+    private final BufferedLogger stepLogger = new BufferedLogger(logger);
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final Map<String, String> params = getParams();
+        final String cubeName = CubingExecutableUtil.getCubeName(params);
+        final String segmentID = CubingExecutableUtil.getSegmentId(params);
+        stepLogger.log("cube: " + cubeName + ", segment: " + segmentID);
+
+        try {
+            CubeManager manager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = manager.getCube(cubeName);
+            CubeSegment segment = cube.getSegmentById(segmentID);
+
+            CubeStatsReader statsReader = new CubeStatsReader(segment, segment.getConfig());
+            Map<Long, Double> cuboidSizeMap = statsReader.getCuboidSizeMap();
+
+            Pair<Integer, Map<Long, Short>> result = calculateCuboidPartitions(
+                    segment.getConfig(), cuboidSizeMap);
+            CuboidShardUtil.saveCuboidShards(segment, result.getSecond(), result.getFirst());
+
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, stepLogger.getBufferedLog());
+
+        } catch (Exception e) {
+            logger.error("CalculateShardsStep failed", e);
+            stepLogger.log("FAILED! " + e.getMessage());
+            return new ExecuteResult(ExecuteResult.State.ERROR, stepLogger.getBufferedLog());
+        }
+    }
+
+    // return (total_shards, cuboid=>num_shards) tuple
+    private Pair<Integer, Map<Long, Short>> calculateCuboidPartitions(
+            KylinConfig cubeConfig, Map<Long, Double> cuboidSizeMap) {
+
+        double totalSizeMB = 0;
+        for (double size : cuboidSizeMap.values()) {
+            totalSizeMB += size;
+        }
+
+        int totalShards = (int) Math.round(totalSizeMB / (cubeConfig.getDruidShardCutGB() * 1024));
+        totalShards = Math.max(totalShards, cubeConfig.getDruidMinShardCount());
+        totalShards = Math.min(totalShards, cubeConfig.getDruidMaxShardCount());
+        stepLogger.log("Estimated total segment size = " + totalSizeMB + " MB, choose " + totalShards + " shards");
+
+        // decide shard count for each cuboid, based on cuboid size
+        Map<Long, Short> cuboidShards = new HashMap<>();
+        final double mbPerShard =  totalSizeMB / totalShards;
+
+        Set<Long> cuboids = new TreeSet<>();
+        cuboids.addAll(cuboidSizeMap.keySet());
+
+        for (Long cuboid : cuboids) {
+            double estimatedSize = cuboidSizeMap.get(cuboid);
+            final double magic = 23;
+            int nShards = (int) (estimatedSize * magic / mbPerShard + 1);
+            nShards = Math.max(nShards, 1);
+            nShards = Math.min(nShards, totalShards);
+
+            cuboidShards.put(cuboid, (short) nShards);
+        }
+
+        return new Pair<>(totalShards, cuboidShards);
+    }
+
+    public void setCubeName(String cubeName) {
+        CubingExecutableUtil.setCubeName(cubeName, getParams());
+    }
+
+    public void setSegmentID(String segmentID) {
+        CubingExecutableUtil.setSegmentId(segmentID, getParams());
+    }
+
+}
\ No newline at end of file
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/ConvertToDruidJob.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/ConvertToDruidJob.java
new file mode 100644
index 0000000000..b467da7fae
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/ConvertToDruidJob.java
@@ -0,0 +1,102 @@
+/*
+ * 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.storage.druid.write;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+
+import java.util.Map;
+
+public class ConvertToDruidJob extends AbstractHadoopJob {
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+        options.addOption(OPTION_JOB_NAME);
+        options.addOption(OPTION_CUBE_NAME);
+        options.addOption(OPTION_SEGMENT_ID);
+        options.addOption(OPTION_INPUT_PATH);
+        options.addOption(OPTION_OUTPUT_PATH);
+        parseOptions(options, args);
+
+        CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        CubeInstance cube = cubeMgr.getCube(getOptionValue(OPTION_CUBE_NAME));
+        CubeSegment segment = cube.getSegmentById(getOptionValue(OPTION_SEGMENT_ID));
+
+        job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
+        setJobClasspath(job, cube.getConfig());
+
+        final Configuration jobConf = job.getConfiguration();
+        jobConf.set(BatchConstants.CFG_CUBE_NAME, cube.getName());
+        jobConf.set(BatchConstants.CFG_CUBE_SEGMENT_ID, segment.getUuid());
+        jobConf.set(BatchConstants.CFG_OUTPUT_PATH, getOptionValue(OPTION_OUTPUT_PATH));
+
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(Text.class);
+        addInputDirs(getOptionValue(OPTION_INPUT_PATH), job);
+
+        job.setPartitionerClass(ShardPartitioner.class);
+        job.setOutputKeyClass(BytesWritable.class);
+        job.setOutputValueClass(Text.class);
+        job.setReducerClass(ConvertToDruidRedcuer.class);
+        job.setNumReduceTasks(segment.getTotalShards());
+
+        final Path outputPath = new Path(getOptionValue(OPTION_OUTPUT_PATH));
+        this.deletePath(jobConf, outputPath);
+        FileOutputFormat.setOutputPath(job, outputPath);
+
+        //allow user specially set config for Druid step
+        for (Map.Entry<String, String> entry : cube.getConfig().getDruidMRConfigOverride().entrySet()) {
+            job.getConfiguration().set(entry.getKey(), entry.getValue());
+        }
+
+        try {
+            // upload segment metadata to distributed cache
+            attachSegmentMetadataWithDict(segment, jobConf);
+            return waitForCompletion(job);
+        } finally {
+            this.cleanupTempConfFile(jobConf);
+        }
+    }
+
+    public static class ShardPartitioner extends Partitioner<Text, Text> {
+        @Override
+        public int getPartition(Text key, Text value, int numPartitions) {
+            short shard = BytesUtil.readShort(key.getBytes(), 0, RowConstants.ROWKEY_SHARDID_LEN);
+            Preconditions.checkState(shard >= 0 && shard < numPartitions, "Invalid shard id %s given %s total shards", shard, numPartitions);
+            return shard;
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/ConvertToDruidRedcuer.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/ConvertToDruidRedcuer.java
new file mode 100644
index 0000000000..a46a8215e7
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/ConvertToDruidRedcuer.java
@@ -0,0 +1,481 @@
+/*
+ * 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.storage.druid.write;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import io.druid.collections.bitmap.WrappedImmutableRoaringBitmap;
+import io.druid.common.guava.ThreadRenamingRunnable;
+import io.druid.concurrent.Execs;
+import io.druid.data.input.InputRow;
+import io.druid.data.input.impl.DimensionsSpec;
+import io.druid.data.input.impl.InputRowParser;
+import io.druid.data.input.impl.MapInputRowParser;
+import io.druid.data.input.impl.TimeAndDimsParseSpec;
+import io.druid.data.input.impl.TimestampSpec;
+import io.druid.java.util.common.IAE;
+import io.druid.java.util.common.granularity.NoneGranularity;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.segment.IndexIO;
+import io.druid.segment.IndexMergerV9;
+import io.druid.segment.IndexSpec;
+import io.druid.segment.QueryableIndex;
+import io.druid.segment.column.ColumnConfig;
+import io.druid.segment.data.CompressionFactory;
+import io.druid.segment.data.RoaringBitmapSerdeFactory;
+import io.druid.segment.incremental.IncrementalIndex;
+import io.druid.segment.incremental.IncrementalIndexSchema;
+import io.druid.timeline.DataSegment;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.CubeDimEncMap;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.RowKeyColDesc;
+import org.apache.kylin.dimension.IDimensionEncodingMap;
+import org.apache.kylin.engine.mr.KylinReducer;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.measure.MeasureCodec;
+import org.apache.kylin.measure.bitmap.RoaringBitmapCounter;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.NameMapping;
+import org.apache.kylin.storage.druid.NameMappingFactory;
+import org.apache.kylin.storage.druid.common.DruidSerdeHelper;
+import org.apache.kylin.storage.druid.common.NumberedShardSpec;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+public class ConvertToDruidRedcuer extends KylinReducer<Text, Text, BytesWritable, Text> {
+    protected static final Logger logger = LoggerFactory.getLogger(ConvertToDruidRedcuer.class);
+    private static final IndexMergerV9 INDEX_MERGER_V9;
+    private static final IndexIO INDEX_IO;
+    private static final ObjectMapper JSON_MAPPER = DruidSerdeHelper.JSON_MAPPER;
+
+    static {
+        INDEX_IO = new IndexIO(JSON_MAPPER, new ColumnConfig() {
+            @Override
+            public int columnCacheSizeBytes() {
+                return 0;
+            }
+        });
+        INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
+
+        //DruidSerdeHelper.registerDruidSerde();
+    }
+
+    private CubeDesc cubeDesc;
+    private CubeSegment cubeSegment;
+    private Map<TblColRef, Integer> dimOrdinals;
+    private int[] dimLens;
+    private MeasureCodec measureCodec;
+    private int[] measureLens;
+    private IDimensionEncodingMap dimEncodings;
+    private DruidSchema cubeSchema;
+    private TimestampSpec timestampSpec;
+    private DimensionsSpec dimensionsSpec;
+    private IndexSpec indexSpec;
+    private InputRowParser<Map<String, Object>> parser;
+    private IncrementalIndex index;
+    private Interval interval;
+    private ListeningExecutorService persistExecutor = null;
+    private List<ListenableFuture<?>> persistFutures = Lists.newArrayList();
+    private File baseFlushFile;
+    private String outputPath;
+    private Set<File> toMerge = Sets.newTreeSet();
+    private int indexCount = 0;
+    private int lineCount = 0;
+    private int runningTotalLineCount = 0;
+
+    private final String druidTimeColumn = "_time_";
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        initKylinMeta(context);
+
+        initDict();
+
+        initDruidConfig(context);
+
+        initFile();
+
+        initBackThread(cubeSegment.getConfig());
+    }
+
+    private void initKylinMeta(Context context) throws IOException {
+        final String cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME);
+        final String segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
+
+        logger.info("cube name = {}, segment id = {}", cubeName, segmentID);
+
+        // KylinConfig was loaded in KylinWriteSupport.init
+        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        CubeManager manager = CubeManager.getInstance(config);
+        CubeInstance cube = manager.getCube(cubeName);
+        cubeSegment = cube.getSegmentById(segmentID);
+        this.cubeDesc = cube.getDescriptor();
+
+        Cuboid cuboid = Cuboid.getBaseCuboid(cubeDesc);
+        NameMapping mapping = NameMappingFactory.getDefault(cubeDesc);
+        cubeSchema = new DruidSchema(mapping, cuboid.getColumns(), cubeDesc.getMeasures());
+    }
+
+    private void initDict() {
+        // determine length of each dimensions
+        dimEncodings = new CubeDimEncMap(cubeSegment);
+        RowKeyColDesc[] rowkeys = cubeDesc.getRowkey().getRowKeyColumns();
+        this.dimOrdinals = new HashMap<>();
+        this.dimLens = new int[rowkeys.length];
+        for (int i = 0; i < rowkeys.length; i++) {
+            TblColRef col = rowkeys[i].getColRef();
+            dimOrdinals.put(col, i);
+            dimLens[i] = dimEncodings.get(col).getLengthOfEncoding();
+        }
+
+        List<MeasureDesc> measures = cubeDesc.getMeasures();
+        this.measureCodec = new MeasureCodec(measures);
+        this.measureLens = new int[measures.size()];
+    }
+
+    private void initDruidConfig(Context context) {
+        timestampSpec = new TimestampSpec(druidTimeColumn, null, null);
+        dimensionsSpec = new DimensionsSpec(cubeSchema.getDimensionSchemas(), null, null);
+        TimeAndDimsParseSpec timeAndDimsParseSpec = new TimeAndDimsParseSpec(timestampSpec, dimensionsSpec);
+        parser = new MapInputRowParser(timeAndDimsParseSpec);
+        indexSpec = new IndexSpec(new RoaringBitmapSerdeFactory(true), null, null, CompressionFactory.LongEncodingStrategy.AUTO);
+        interval = DruidSchema.segmentInterval(cubeSegment);
+        index = makeIncrementalIndex(cubeSchema.getAggregators());
+
+        outputPath = context.getConfiguration().get(BatchConstants.CFG_OUTPUT_PATH);
+    }
+
+    private void initFile() throws IOException {
+        baseFlushFile = File.createTempFile("base", "flush");
+        baseFlushFile.delete();
+        baseFlushFile.mkdirs();
+    }
+
+    private void initBackThread(KylinConfig config) {
+        int numBackgroundPersistThreads = config.getDruidReducerThreadsNum();
+        logger.info("numBackgroundPersistThreads: " + numBackgroundPersistThreads);
+        if (numBackgroundPersistThreads > 0) {
+            final BlockingQueue<Runnable> queue = new SynchronousQueue<>();
+            ExecutorService executorService = new ThreadPoolExecutor(numBackgroundPersistThreads, numBackgroundPersistThreads, 0L, TimeUnit.MILLISECONDS, queue, Execs.makeThreadFactory("IndexGeneratorJob_persist_%d"), new RejectedExecutionHandler() {
+                @Override
+                public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
+                    try {
+                        executor.getQueue().put(r);
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                        throw new RejectedExecutionException("Got Interrupted while adding to the Queue");
+                    }
+                }
+            });
+            persistExecutor = MoreExecutors.listeningDecorator(executorService);
+        } else {
+            persistExecutor = MoreExecutors.sameThreadExecutor();
+        }
+    }
+
+    @Override
+    protected void doReduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+        long startTime = System.currentTimeMillis();
+
+        for (Text value : values) {
+            Map<String, Object> druidEvent = new HashMap<>();
+            Cuboid cuboid = getCuboid(key);
+
+            druidEvent.put(druidTimeColumn, interval.getStart());
+
+            writeCuboid(cuboid, druidEvent);
+            writeDimensions(key, cuboid, druidEvent);
+            writeMeasures(value, druidEvent);
+
+            InputRow druidRow = parser.parse(druidEvent);
+
+            int numRows = index.add(druidRow);
+
+            ++lineCount;
+
+            if (!index.canAppendRow()) {
+                logger.info(index.getOutOfRowsReason());
+                logger.info("{} lines to {} rows in {} millis", lineCount - runningTotalLineCount, numRows, System.currentTimeMillis() - startTime);
+
+                runningTotalLineCount = lineCount;
+
+                final File file = new File(baseFlushFile, String.format(Locale.ROOT, "index%05d", indexCount));
+                toMerge.add(file);
+
+                context.progress();
+
+                final IncrementalIndex persistIndex = index;
+                persistFutures.add(persistExecutor.submit(new ThreadRenamingRunnable(String.format(Locale.ROOT, "%s-persist", file.getName())) {
+                    @Override
+                    public void doRun() {
+                        try {
+                            INDEX_MERGER_V9.persist(persistIndex, interval, file, indexSpec);
+                        } catch (Exception e) {
+                            logger.error("persist index error", e);
+                            throw Throwables.propagate(e);
+                        } finally {
+                            // close this index
+                            persistIndex.close();
+                        }
+                    }
+                }));
+
+                index = makeIncrementalIndex(cubeSchema.getAggregators());
+                startTime = System.currentTimeMillis();
+                ++indexCount;
+            }
+        }
+    }
+
+    private Cuboid getCuboid(Text key) {
+        long id = Bytes.toLong(key.getBytes(), RowConstants.ROWKEY_SHARDID_LEN, RowConstants.ROWKEY_CUBOIDID_LEN);
+        return Cuboid.findById(cubeDesc, id);
+    }
+
+    private void writeCuboid(Cuboid cuboid, Map<String, Object> druidEvent) {
+        druidEvent.put(DruidSchema.ID_COL, Long.toString(cuboid.getId()));
+    }
+
+    private void writeDimensions(Text key, Cuboid cuboid, Map<String, Object> druidEvent) {
+        int offset = RowConstants.ROWKEY_SHARD_AND_CUBOID_LEN;
+
+        for (TblColRef dim : cuboid.getColumns()) {
+            int len = dimLens[dimOrdinals.get(dim)];
+            String rowValue = dimEncodings.get(dim).decode(key.getBytes(), offset, len);
+            druidEvent.put(cubeSchema.getDimFieldName(dim), rowValue);
+            offset += len;
+        }
+    }
+
+    private void writeMeasures(Text value, Map<String, Object> druidEvent) {
+        // determine offsets and lengths of each measure
+        ByteBuffer valBuf = ByteBuffer.wrap(value.getBytes(), 0, value.getLength());
+        measureCodec.getPeekLength(valBuf, measureLens);
+
+        int i = 0, offset = 0;
+        for (MeasureDesc measure : cubeDesc.getMeasures()) {
+            int len = measureLens[i];
+            Object data = measureCodec.decode(ByteBuffer.wrap(value.getBytes(), offset, len), i);
+
+            if (data instanceof RoaringBitmapCounter) {
+                RoaringBitmapCounter counter = (RoaringBitmapCounter) data;
+                data = new WrappedImmutableRoaringBitmap(counter.getImmutableBimap());
+            }
+
+            // for extend-column measure
+            if (data instanceof ByteArray) {
+                ByteArray array = (ByteArray) data;
+                //the byte array for extend-column maybe NULL
+                if (array.array() == null) {
+                    data = new String(new byte[] {}, Charset.forName("UTF-8"));
+                } else {
+                    data = Bytes.toString(array.array());
+                }
+            }
+
+            druidEvent.put(cubeSchema.getMeasureFieldName(measure), data);
+
+            i++;
+            offset += len;
+        }
+    }
+
+    @Override
+    protected void doCleanup(Context context) throws IOException, InterruptedException {
+        if (lineCount == 0) {
+            logger.info("The input for this reducer is zero!");
+            return;
+        }
+
+        int taskId = context.getTaskAttemptID().getTaskID().getId();
+
+        try {
+            logger.info("{} lines completed.", lineCount);
+
+            List<QueryableIndex> indexes = Lists.newArrayListWithCapacity(indexCount);
+            final File mergedBase;
+
+            if (toMerge.size() == 0) {
+                if (index.isEmpty()) {
+                    throw new IAE("If you try to persist empty indexes you are going to have a bad time");
+                }
+
+                mergedBase = new File(baseFlushFile, "merged");
+                INDEX_MERGER_V9.persist(index, interval, mergedBase, indexSpec);
+            } else {
+                if (!index.isEmpty()) {
+                    final File finalFile = new File(baseFlushFile, "final");
+                    INDEX_MERGER_V9.persist(index, interval, finalFile, indexSpec);
+                    toMerge.add(finalFile);
+                }
+
+                Futures.allAsList(persistFutures).get(1, TimeUnit.HOURS);
+                persistExecutor.shutdown();
+
+                for (File file : toMerge) {
+                    indexes.add(INDEX_IO.loadIndex(file));
+                }
+
+                logger.info("starting merge of intermediate persisted segments.");
+                long mergeStartTime = System.currentTimeMillis();
+                mergedBase = INDEX_MERGER_V9.mergeQueryableIndex(indexes, false, cubeSchema.getAggregators(), new File(baseFlushFile, "merged"), indexSpec);
+                logger.info("finished merge of intermediate persisted segments. time taken {} ms.", (System.currentTimeMillis() - mergeStartTime));
+
+            }
+
+            Pair<Path, Long> results = zipAndUploadFile(context, mergedBase, outputPath, taskId);
+
+            DataSegment segment = getDataSegment(cubeSegment, results.getFirst(), taskId, results.getSecond());
+            writeSegmentDescriptor(context, segment, outputPath, taskId);
+        } catch (ExecutionException | TimeoutException e) {
+            throw Throwables.propagate(e);
+        }
+    }
+
+    private final int fsBufferSize = 1 << 18;
+
+    private Pair<Path, Long> zipAndUploadFile(Context context, File localFile, String outputPath, int shardId) throws IOException, InterruptedException {
+        long totalSize = 0;
+
+        // zip & uploading to hdfs
+        Path zipPath = new Path(outputPath, shardId + ".zip");
+        FileSystem fs = zipPath.getFileSystem(context.getConfiguration());
+        logger.info("uploading index to {}", zipPath);
+
+        try (OutputStream os = fs.create(zipPath, true, fsBufferSize, context); ZipOutputStream out = new ZipOutputStream(os)) {
+            File[] files = localFile.listFiles();
+            for (File file : files) {
+                if (Files.isRegularFile(file.toPath())) {
+                    ZipEntry entry = new ZipEntry(file.getName());
+                    out.putNextEntry(entry);
+                    long size = Files.copy(file.toPath(), out);
+                    context.progress();
+                    logger.info("Added ZipEntry[{}] of {} bytes", entry.getName(), size);
+                    out.closeEntry();
+
+                    totalSize += size;
+                }
+            }
+            out.flush();
+        }
+
+        logger.info("total size: " + totalSize);
+        return new Pair<>(zipPath, totalSize);
+    }
+
+    private void writeSegmentDescriptor(Context context, DataSegment segment, String outputPath, int shardId) throws IOException {
+        Path descriptorPath = new Path(outputPath, shardId + "-" + "descriptor");
+        FileSystem outputFS = descriptorPath.getFileSystem(context.getConfiguration());
+
+        if (outputFS.exists(descriptorPath)) {
+            if (!outputFS.delete(descriptorPath, false)) {
+                throw new IOException(String.format(Locale.ROOT, "Failed to delete descriptor at [%s]", descriptorPath));
+            }
+        }
+
+        logger.info("will write " + segment + " to " + descriptorPath);
+
+        try (final OutputStream descriptorOut = outputFS.create(descriptorPath, true, fsBufferSize)) {
+            JSON_MAPPER.writeValue(descriptorOut, segment);
+            descriptorOut.flush();
+        }
+
+        logger.info("write " + segment + " done");
+    }
+
+    private DataSegment getDataSegment(CubeSegment segment, Path shardPath, int shardId, long size) {
+        ImmutableMap<String, Object> loadSpec = ImmutableMap.<String, Object> of("type", "hdfs", "path", shardPath.toUri().toString());
+
+        int shardNum = segment.getTotalShards();
+
+        NumberedShardSpec shardSpec = new NumberedShardSpec(shardId, shardNum);
+
+        List<String> dimensionNames = new ArrayList<>();
+        dimensionNames.add(DruidSchema.ID_COL);
+        for (TblColRef dim : cubeSchema.getDimensions()) {
+            dimensionNames.add(cubeSchema.getDimFieldName(dim));
+        }
+
+        List<String> measureNames = new ArrayList<>();
+        for (MeasureDesc met : cubeSchema.getMeasures()) {
+            measureNames.add(cubeSchema.getMeasureFieldName(met));
+        }
+
+        return new DataSegment(DruidSchema.getDataSource(segment.getCubeDesc()), interval, segment.getCreateTimeUTCStr(), loadSpec, dimensionNames, measureNames, shardSpec, 9, size);
+    }
+
+    private IncrementalIndex makeIncrementalIndex(AggregatorFactory[] metrics) {
+        final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder().withTimestampSpec(timestampSpec).withMinTimestamp(interval.getStartMillis()).withDimensionsSpec(dimensionsSpec).withQueryGranularity(new NoneGranularity()).withMetrics(metrics).withRollup(false).build();
+
+        return new IncrementalIndex.Builder()
+                .setIndexSchema(indexSchema)
+                .setReportParseExceptions(true)
+                .setConcurrentEventAdd(true)
+                .setMaxRowCount(100000)
+                .buildOnheap();
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/DruidMROutput.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/DruidMROutput.java
new file mode 100644
index 0000000000..87ce53dbde
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/DruidMROutput.java
@@ -0,0 +1,294 @@
+/*
+ * 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.storage.druid.write;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.engine.mr.IMROutput2;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.common.MapReduceUtil;
+import org.apache.kylin.engine.mr.steps.HiveToBaseCuboidMapper;
+import org.apache.kylin.engine.mr.steps.InMemCuboidMapper;
+import org.apache.kylin.engine.mr.steps.MergeCuboidJob;
+import org.apache.kylin.engine.mr.steps.NDCuboidMapper;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.storage.druid.DruidSchema;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.kylin.engine.mr.JobBuilderSupport.appendExecCmdParameters;
+
+public class DruidMROutput implements IMROutput2 {
+
+    @Override
+    public IMRBatchCubingOutputSide2 getBatchCubingOutputSide(final CubeSegment seg) {
+        return new IMRBatchCubingOutputSide2() {
+            private JobBuilderSupport support = new JobBuilderSupport(seg, "");
+
+            @Override
+            public void addStepPhase2_BuildDictionary(DefaultChainedExecutable jobFlow) {
+                createCalculateShardsStep(seg, jobFlow);
+                createUpdateDruidTierStep(seg, jobFlow);
+            }
+
+            @Override
+            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow) {
+                createConvertCuboidToDruidStep(support, seg, jobFlow);
+            }
+
+            @Override
+            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+
+            }
+
+            @Override
+            public IMROutputFormat getOuputFormat() {
+                return new DruidMROutputFormat();
+            }
+
+        };
+    }
+
+    @Override
+    public IMRBatchMergeOutputSide2 getBatchMergeOutputSide(final CubeSegment seg) {
+        return new IMRBatchMergeOutputSide2() {
+            private JobBuilderSupport support = new JobBuilderSupport(seg, "");
+
+            @Override
+            public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow) {
+                createCalculateShardsStep(seg, jobFlow);
+                createUpdateDruidTierStep(seg, jobFlow);
+            }
+
+            @Override
+            public void addStepPhase2_BuildCube(CubeSegment seg, List<CubeSegment> mergingSegments, DefaultChainedExecutable jobFlow) {
+                jobFlow.addTask(createMergeCuboidDataStep(support, seg, mergingSegments, jobFlow.getId(), MergeCuboidJob.class));
+
+                createConvertCuboidToDruidStep(support, seg, jobFlow);
+            }
+
+            @Override
+            public void addStepPhase3_Cleanup(DefaultChainedExecutable jobFlow) {
+                addMergingGarbageCollectionSteps(support, seg, jobFlow);
+            }
+
+            @Override
+            public IMRMergeOutputFormat getOuputFormat() {
+                return new DruidMergeMROutputFormat();
+            }
+        };
+    }
+
+    //Todo KOD support IMRBatchOptimizeOutputSide2
+    public IMRBatchOptimizeOutputSide2 getBatchOptimizeOutputSide(final CubeSegment seg) {
+        return new IMRBatchOptimizeOutputSide2() {
+
+            @Override
+            public void addStepPhase2_CreateHTable(DefaultChainedExecutable jobFlow) {
+            }
+
+            @Override
+            public void addStepPhase3_BuildCube(DefaultChainedExecutable jobFlow) {
+            }
+
+            @Override
+            public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+            }
+
+            @Override
+            public void addStepPhase5_Cleanup(DefaultChainedExecutable jobFlow) {
+            }
+        };
+    }
+
+    private void createCalculateShardsStep(CubeSegment seg, DefaultChainedExecutable jobFlow) {
+        CalculateShardsStep step = new CalculateShardsStep();
+        step.setName("Calculate Shards Info");
+        step.setCubeName(seg.getRealization().getName());
+        step.setSegmentID(seg.getUuid());
+        jobFlow.addTask(step);
+    }
+
+    private void createUpdateDruidTierStep(CubeSegment seg, DefaultChainedExecutable jobFlow) {
+        UpdateDruidTierStep step = new UpdateDruidTierStep();
+        step.setName("Update Druid Tier");
+        step.setCubeName(seg.getRealization().getName());
+        jobFlow.addTask(step);
+    }
+
+    private void createConvertCuboidToDruidStep(JobBuilderSupport support, CubeSegment seg, DefaultChainedExecutable jobFlow) {
+        final String cubeName = seg.getRealization().getName();
+        final String dataSource = DruidSchema.getDataSource(seg.getCubeDesc());
+
+        final String inputPath = support.getCuboidRootPath(jobFlow.getId()) + "*";
+        final String outputPath = seg.getConfig().getDruidHdfsLocation() + "/" + dataSource + "/" + seg.getUuid();
+
+        StringBuilder cmd = new StringBuilder();
+        support.appendMapReduceParameters(cmd);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Druid_Generator_" + seg + "_Step");
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, cubeName);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, inputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
+
+        MapReduceExecutable step = new MapReduceExecutable();
+        step.setName("Convert Cuboid to Druid");
+        step.setMapReduceJobClass(ConvertToDruidJob.class);
+        step.setMapReduceParams(cmd.toString());
+
+        jobFlow.addTask(step);
+
+        LoadDruidSegmentStep step2 = new LoadDruidSegmentStep();
+        step2.setName("Load Segment to Druid");
+        step2.setCubeName(seg.getRealization().getName());
+        step2.setSegmentID(seg.getUuid());
+        jobFlow.addTask(step2);
+    }
+
+    private MapReduceExecutable createMergeCuboidDataStep(JobBuilderSupport support, CubeSegment seg, List<CubeSegment> mergingSegments, String jobID, Class<? extends AbstractHadoopJob> clazz) {
+        final List<String> mergingCuboidPaths = Lists.newArrayList();
+        for (CubeSegment merging : mergingSegments) {
+            mergingCuboidPaths.add(support.getCuboidRootPath(merging) + "*");
+        }
+        String formattedPath = StringUtil.join(mergingCuboidPaths, ",");
+        String outputPath = support.getCuboidRootPath(jobID);
+
+        MapReduceExecutable mergeCuboidDataStep = new MapReduceExecutable();
+        mergeCuboidDataStep.setName(ExecutableConstants.STEP_NAME_MERGE_CUBOID);
+        StringBuilder cmd = new StringBuilder();
+
+        support.appendMapReduceParameters(cmd);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getCubeInstance().getName());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+        appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, formattedPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
+        appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Merge_Cuboid_" + seg.getCubeInstance().getName() + "_Step");
+
+        mergeCuboidDataStep.setMapReduceParams(cmd.toString());
+        mergeCuboidDataStep.setMapReduceJobClass(clazz);
+        return mergeCuboidDataStep;
+    }
+
+    private void addMergingGarbageCollectionSteps(JobBuilderSupport support, CubeSegment seg, DefaultChainedExecutable jobFlow) {
+        String jobId = jobFlow.getId();
+
+        List<String> toDeletePaths = new ArrayList<>();
+        toDeletePaths.addAll(getMergingHDFSPaths(support, seg));
+
+        HDFSPathGCStep step = new HDFSPathGCStep();
+        step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION_HDFS);
+        step.setDeletePaths(toDeletePaths);
+        step.setJobId(jobId);
+
+        jobFlow.addTask(step);
+    }
+
+    private List<String> getMergingHDFSPaths(JobBuilderSupport support, CubeSegment seg) {
+        final List<CubeSegment> mergingSegments = ((CubeInstance) seg.getRealization()).getMergingSegments((CubeSegment) seg);
+        Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge, target segment " + seg);
+        final List<String> mergingHDFSPaths = Lists.newArrayList();
+        for (CubeSegment merging : mergingSegments) {
+            //kylin cuboid data
+            mergingHDFSPaths.add(support.getJobWorkingDir(merging.getLastBuildJobID()));
+        }
+        return mergingHDFSPaths;
+    }
+
+
+
+    public static class DruidMROutputFormat implements IMROutputFormat {
+
+        @Override
+        public void configureJobInput(Job job, String input) throws Exception {
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+        }
+
+        @Override
+        public void configureJobOutput(Job job, String output, CubeSegment segment, CuboidScheduler cuboidScheduler,
+                                       int level) throws Exception {
+            int reducerNum = 1;
+            Class mapperClass = job.getMapperClass();
+
+            //allow user specially set config for base cuboid step
+            if (mapperClass == HiveToBaseCuboidMapper.class) {
+                for (Map.Entry<String, String> entry : segment.getConfig().getBaseCuboidMRConfigOverride().entrySet()) {
+                    job.getConfiguration().set(entry.getKey(), entry.getValue());
+                }
+            }
+
+            if (mapperClass == HiveToBaseCuboidMapper.class || mapperClass == NDCuboidMapper.class) {
+                reducerNum = MapReduceUtil.getLayeredCubingReduceTaskNum(segment, cuboidScheduler,
+                        AbstractHadoopJob.getTotalMapInputMB(job), level);
+            } else if (mapperClass == InMemCuboidMapper.class) {
+                reducerNum = MapReduceUtil.getInmemCubingReduceTaskNum(segment, cuboidScheduler);
+            }
+            Path outputPath = new Path(output);
+            FileOutputFormat.setOutputPath(job, outputPath);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+            job.setNumReduceTasks(reducerNum);
+            HadoopUtil.deletePath(job.getConfiguration(), outputPath);
+        }
+    }
+
+
+    public static class DruidMergeMROutputFormat implements IMRMergeOutputFormat {
+
+        @Override
+        public void configureJobInput(Job job, String input) throws Exception {
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+        }
+
+        @Override
+        public void configureJobOutput(Job job, String output, CubeSegment segment) throws Exception {
+            int reducerNum = MapReduceUtil.getLayeredCubingReduceTaskNum(segment, segment.getCuboidScheduler(),
+                    AbstractHadoopJob.getTotalMapInputMB(job), -1);
+            job.setNumReduceTasks(reducerNum);
+
+            Path outputPath = new Path(output);
+            HadoopUtil.deletePath(job.getConfiguration(), outputPath);
+            FileOutputFormat.setOutputPath(job, outputPath);
+            job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        }
+
+        @Override
+        public CubeSegment findSourceSegment(FileSplit fileSplit, CubeInstance cube) {
+            String filePath = fileSplit.getPath().toString();
+            String jobID = JobBuilderSupport.extractJobIDFromPath(filePath);
+            return CubeInstance.findSegmentWithJobId(jobID, cube);
+        }
+
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/HDFSPathGCStep.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/HDFSPathGCStep.java
new file mode 100644
index 0000000000..7b16ca622c
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/HDFSPathGCStep.java
@@ -0,0 +1,131 @@
+/*
+ * 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.storage.druid.write;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class HDFSPathGCStep extends AbstractExecutable {
+    private static final Logger logger = LoggerFactory.getLogger(HDFSPathGCStep.class);
+
+    private static final String TO_DELETE_PATHS = "toDeletePaths";
+    private StringBuffer output;
+    private JobEngineConfig config;
+
+    public HDFSPathGCStep() {
+        super();
+        output = new StringBuffer();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        try {
+            config = new JobEngineConfig(context.getConfig());
+            List<String> toDeletePaths = getDeletePaths();
+            dropHdfsPathOnCluster(toDeletePaths, HadoopUtil.getWorkingFileSystem());
+        } catch (IOException e) {
+            logger.error("job:" + getId() + " execute finished with exception", e);
+            output.append("\n").append(e.getLocalizedMessage());
+            return new ExecuteResult(ExecuteResult.State.ERROR, output.toString());
+        }
+
+        return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
+    }
+
+    private void dropHdfsPathOnCluster(List<String> oldHdfsPaths, FileSystem fileSystem) throws IOException {
+        if (oldHdfsPaths != null && oldHdfsPaths.size() > 0) {
+            logger.debug("Drop HDFS path on FileSystem: " + fileSystem.getUri());
+            output.append("Drop HDFS path on FileSystem: \"" + fileSystem.getUri() + "\" \n");
+            for (String path : oldHdfsPaths) {
+                if (path.endsWith("*"))
+                    path = path.substring(0, path.length() - 1);
+
+                Path oldPath = new Path(path);
+                if (fileSystem.exists(oldPath)) {
+                    fileSystem.delete(oldPath, true);
+                    logger.debug("HDFS path " + path + " is dropped.");
+                    output.append("HDFS path " + path + " is dropped.\n");
+                } else {
+                    logger.debug("HDFS path " + path + " not exists.");
+                    output.append("HDFS path " + path + " not exists.\n");
+                }
+                // If hbase was deployed on another cluster, the job dir is empty and should be dropped,
+                // because of rowkey_stats and hfile dirs are both dropped.
+                if (fileSystem.listStatus(oldPath.getParent()).length == 0) {
+                    Path emptyJobPath = new Path(JobBuilderSupport.getJobWorkingDir(config, getJobId()));
+                    if (fileSystem.exists(emptyJobPath)) {
+                        fileSystem.delete(emptyJobPath, true);
+                        logger.debug("HDFS path " + emptyJobPath + " is empty and dropped.");
+                        output.append("HDFS path " + emptyJobPath + " is empty and dropped.\n");
+                    }
+                }
+            }
+        }
+    }
+
+    void setDeletePaths(List<String> deletePaths) {
+        setArrayParam(TO_DELETE_PATHS, deletePaths);
+    }
+
+    void setJobId(String jobId) {
+        setParam("jobId", jobId);
+    }
+
+    private List<String> getDeletePaths() {
+        return getArrayParam(TO_DELETE_PATHS);
+    }
+
+    private String getJobId() {
+        return getParam("jobId");
+    }
+
+    private void setArrayParam(String paramKey, List<String> paramValues) {
+        setParam(paramKey, StringUtils.join(paramValues, ","));
+    }
+
+    private List<String> getArrayParam(String paramKey) {
+        final String ids = getParam(paramKey);
+        if (ids != null) {
+            final String[] splitted = StringUtils.split(ids, ",");
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
+            for (String id : splitted) {
+                result.add(id);
+            }
+            return result;
+        } else {
+            return Collections.emptyList();
+        }
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/LoadDruidSegmentStep.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/LoadDruidSegmentStep.java
new file mode 100644
index 0000000000..7d77efd054
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/LoadDruidSegmentStep.java
@@ -0,0 +1,168 @@
+/*
+ * 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.storage.druid.write;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.kylin.storage.druid.common.DruidSerdeHelper.JSON_MAPPER;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BufferedLogger;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.common.DruidCoordinatorClient;
+import org.apache.kylin.storage.druid.common.DruidServerMetadata;
+import org.apache.kylin.storage.druid.common.ImmutableSegmentLoadInfo;
+import org.joda.time.Interval;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import io.druid.java.util.common.ISE;
+import io.druid.timeline.DataSegment;
+
+public class LoadDruidSegmentStep extends AbstractExecutable {
+    private static final long MAX_LOAD_MILLIS = 2 * 3600 * 1000;
+
+    private final BufferedLogger stepLogger = new BufferedLogger(logger);
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final Map<String, String> params = getParams();
+        final String cubeName = CubingExecutableUtil.getCubeName(params);
+        final String segmentID = CubingExecutableUtil.getSegmentId(params);
+        stepLogger.log("cube: " + cubeName + ", segment: " + segmentID);
+
+        try {
+            CubeManager manager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = manager.getCube(cubeName);
+            CubeSegment segment = cube.getSegmentById(segmentID);
+
+            final String dataSource = DruidSchema.getDataSource(segment.getCubeDesc());
+
+            long start = System.currentTimeMillis();
+            final Set<DataSegment> druidSegments = getDruidSegmentDescriptors(dataSource, segment);
+            new AnnounceDruidSegment().announceHistoricalSegments(druidSegments);
+            stepLogger.log("Read SegmentDescriptors and update druid mysql metadata elapsed time: "
+                    + (System.currentTimeMillis() - start));
+
+            start = System.currentTimeMillis();
+            boolean success = false;
+            final DruidCoordinatorClient coordinatorClient = DruidCoordinatorClient.getSingleton();
+
+            while (System.currentTimeMillis() < (start + MAX_LOAD_MILLIS)) {
+                int loaded = numLoadedDruidSegments(coordinatorClient, segment);
+                if (loaded == druidSegments.size()) {
+                    success = true;
+                    break;
+                }
+                logger.info("Still waiting for druid to load segments ({}/{} loaded)", loaded, druidSegments.size());
+                Thread.sleep(60000);
+            }
+
+            if (!success) {
+                throw new ISE("Timeout after %,dms", MAX_LOAD_MILLIS);
+            }
+
+            stepLogger.log(StringUtils.format("Successfully loaded %d segments in %,dms", druidSegments.size(),
+                    System.currentTimeMillis() - start));
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, stepLogger.getBufferedLog());
+
+        } catch (Exception e) {
+            logger.error("LoadDruidSegmentStep failed: ", e);
+            stepLogger.log("LoadDruidSegmentStep failed " + e.getMessage());
+            return new ExecuteResult(ExecuteResult.State.ERROR, stepLogger.getBufferedLog());
+        }
+    }
+
+    private int numLoadedDruidSegments(DruidCoordinatorClient coordinatorClient, CubeSegment segment) {
+        final String dataSource = DruidSchema.getDataSource(segment.getCubeDesc());
+        final Interval interval = DruidSchema.segmentInterval(segment);
+        final String version = segment.getCreateTimeUTCStr();
+
+        List<ImmutableSegmentLoadInfo> loaded = coordinatorClient.fetchServerView(dataSource, interval);
+
+        int count = 0;
+        for (ImmutableSegmentLoadInfo info : loaded) {
+            if (info.getSegment().getVersion().compareTo(version) >= 0
+                    && Iterables.any(info.getServers(), new Predicate<DruidServerMetadata>() {
+                        @Override
+                        public boolean apply(DruidServerMetadata input) {
+                            return "historical".equals(input.getType());
+                        }
+                    })) {
+                count++;
+            }
+        }
+        return count;
+    }
+
+    private Set<DataSegment> getDruidSegmentDescriptors(String dataSource, CubeSegment cubeSegment) throws IOException {
+        Set<DataSegment> segmentSet = new HashSet<>();
+        FileSystem fs = HadoopUtil.getWorkingFileSystem();
+
+        long totalSize = 0;
+        for (int i = 0; i < cubeSegment.getTotalShards(); i++) {
+            Path descriptorPath = getSegmentDescriptor(dataSource, cubeSegment, i);
+            try (FSDataInputStream in = fs.open(descriptorPath)) {
+                DataSegment dataSegment = JSON_MAPPER.readValue((InputStream) in, DataSegment.class);
+                segmentSet.add(dataSegment);
+                totalSize += dataSegment.getSize();
+            }
+        }
+
+        checkArgument(segmentSet.size() == cubeSegment.getTotalShards(), "expect %d value, got %d",
+                cubeSegment.getTotalShards(), segmentSet.size());
+
+        addExtraInfo(CubingJob.CUBE_SIZE_BYTES, totalSize + "");
+        return segmentSet;
+    }
+
+    private Path getSegmentDescriptor(String dataSource, CubeSegment segment, int shardId) {
+        return new Path(segment.getConfig().getDruidHdfsLocation() + "/" + dataSource + "/" + segment.getUuid(),
+                shardId + "-" + "descriptor");
+    }
+
+    public void setCubeName(String cubeName) {
+        CubingExecutableUtil.setCubeName(cubeName, getParams());
+    }
+
+    public void setSegmentID(String segmentID) {
+        CubingExecutableUtil.setSegmentId(segmentID, getParams());
+    }
+}
diff --git a/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/UpdateDruidTierStep.java b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/UpdateDruidTierStep.java
new file mode 100644
index 0000000000..a2fd620940
--- /dev/null
+++ b/storage-druid/src/main/java/org/apache/kylin/storage/druid/write/UpdateDruidTierStep.java
@@ -0,0 +1,89 @@
+/*
+ * 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.storage.druid.write;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.BufferedLogger;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.storage.druid.DruidSchema;
+import org.apache.kylin.storage.druid.common.DruidCoordinatorClient;
+import org.apache.kylin.storage.druid.common.DruidSerdeHelper;
+import org.apache.kylin.storage.druid.common.ForeverLoadRule;
+import org.apache.kylin.storage.druid.common.Rule;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import io.druid.java.util.common.StringUtils;
+
+public class UpdateDruidTierStep extends AbstractExecutable {
+    private final BufferedLogger stepLogger = new BufferedLogger(logger);
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final Map<String, String> params = getParams();
+        final String cubeName = CubingExecutableUtil.getCubeName(params);
+        final DruidCoordinatorClient coordinatorClient = DruidCoordinatorClient.getSingleton();
+
+        try {
+            CubeManager manager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = manager.getCube(cubeName);
+            final String dataSource = DruidSchema.getDataSource(cube.getDescriptor());
+
+            List<Rule> newRules = getRulesForCube(cube);
+            List<Rule> oldRules = coordinatorClient.getRules(dataSource);
+
+            if (newRules.equals(oldRules)) {
+                stepLogger.log("Current rules for datasource " + dataSource + " is up to date, done");
+            } else {
+                stepLogger.log(StringUtils.format("Setting new rules for datasource %s : %s", dataSource,
+                        DruidSerdeHelper.JSON_MAPPER.writeValueAsString(newRules)));
+                coordinatorClient.putRules(dataSource, newRules);
+                stepLogger.log("Done");
+            }
+
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, stepLogger.getBufferedLog());
+
+        } catch (Exception e) {
+            logger.error("UpdateDruidTierStep failed", e);
+            stepLogger.log("FAILED! " + e.getMessage());
+            return new ExecuteResult(ExecuteResult.State.ERROR, stepLogger.getBufferedLog());
+        }
+    }
+
+    private List<Rule> getRulesForCube(CubeInstance cube) {
+        // TODO consider cube retention
+        Map<String, Integer> tieredReplicants = Maps.newHashMap();
+        tieredReplicants.put(cube.getConfig().getDruidTierName(), cube.getConfig().getDruidReplicationNum());
+        return Lists.<Rule> newArrayList(new ForeverLoadRule(tieredReplicants));
+    }
+
+    public void setCubeName(String cubeName) {
+        CubingExecutableUtil.setCubeName(cubeName, getParams());
+    }
+}
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeStorageQuery.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeStorageQuery.java
index fe483ba781..6456c8ca38 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeStorageQuery.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeStorageQuery.java
@@ -42,4 +42,9 @@ protected String getGTStorage() {
             return KylinConfig.getInstanceFromEnv().getDefaultIGTStorage();
         }
     }
+
+    @Override
+    public boolean keepRuntimeFilter() {
+        return true;
+    }
 }
diff --git a/tool/pom.xml b/tool/pom.xml
index 958bd55ba4..942be9b03d 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -63,6 +63,24 @@
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-server-base</artifactId>
         </dependency>
+
+
+        <dependency>
+            <groupId>io.airlift</groupId>
+            <artifactId>airline</artifactId>
+            <version>0.7</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>annotations</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
         <!--Env-->
         <dependency>
             <groupId>org.apache.hbase</groupId>
diff --git a/tool/src/main/java/org/apache/kylin/tool/DruidCubeMigrationCLI.java b/tool/src/main/java/org/apache/kylin/tool/DruidCubeMigrationCLI.java
new file mode 100644
index 0000000000..abf8426fb3
--- /dev/null
+++ b/tool/src/main/java/org/apache/kylin/tool/DruidCubeMigrationCLI.java
@@ -0,0 +1,553 @@
+/*
+ * 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.tool;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.StorageURL;
+import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.RawResource;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.Serializer;
+import org.apache.kylin.common.restclient.RestClient;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeDescManager;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.dict.DictionaryInfo;
+import org.apache.kylin.dict.DictionaryManager;
+import org.apache.kylin.dict.lookup.SnapshotManager;
+import org.apache.kylin.dict.lookup.SnapshotTable;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TableExtDesc;
+import org.apache.kylin.metadata.model.TableRef;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.realization.RealizationStatusEnum;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.storage.hbase.HBaseConnection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DruidCubeMigrationCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(DruidCubeMigrationCLI.class);
+
+    private List<Opt> operations;
+    protected KylinConfig srcConfig;
+    protected KylinConfig dstConfig;
+    private ResourceStore srcStore;
+    private ResourceStore dstStore;
+    private FileSystem hdfsFS;
+
+    public static final String ACL_INFO_FAMILY = "i";
+    private static final String ACL_TABLE_NAME = "_acl";
+    private static final String ACL_INFO_FAMILY_PARENT_COLUMN = "p";
+
+    public static void main(String[] args) throws IOException, InterruptedException {
+
+        DruidCubeMigrationCLI cli = new DruidCubeMigrationCLI();
+        if (args.length != 8) {
+            cli.usage();
+            System.exit(1);
+        }
+        cli.moveCube(args[0], args[1], args[2], args[3], args[4], args[5], args[6], args[7]);
+    }
+
+    protected void usage() {
+        System.out.println("Usage: DruidCubeMigrationCLI srcKylinConfigUri dstKylinConfigUri cubeName projectName copyAclOrNot purgeOrNot overwriteIfExists realExecute");
+        System.out.println(" srcKylinConfigUri: The KylinConfig of the cube’s source \n" + "dstKylinConfigUri: The KylinConfig of the cube’s new home \n" + "cubeName: the name of cube to be migrated. \n" + "projectName: The target project in the target environment.(Make sure it exist) \n" + "copyAclOrNot: true or false: whether copy cube ACL to target environment. \n" + "purgeOrNot: true or false: whether purge the cube from src server after the migration. \n" + "overwriteIfExists: overwrite cube if it already exists in the target environment. \n" + "realExecute: if false, just print the operations to take, if true, do the real migration. \n");
+
+    }
+
+    public void moveCube(KylinConfig srcCfg, KylinConfig dstCfg, String cubeName, String projectName, String copyAcl, String purgeAndDisable, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
+
+        srcConfig = srcCfg;
+        srcStore = ResourceStore.getStore(srcConfig);
+        dstConfig = dstCfg;
+        dstStore = ResourceStore.getStore(dstConfig);
+
+        CubeManager cubeManager = CubeManager.getInstance(srcConfig);
+        CubeInstance cube = cubeManager.getCube(cubeName);
+        logger.info("cube to be moved is : " + cubeName);
+
+        if (cube.getStatus() != RealizationStatusEnum.READY)
+            throw new IllegalStateException("Cannot migrate cube that is not in READY state.");
+
+        for (CubeSegment segment : cube.getSegments()) {
+            if (segment.getStatus() != SegmentStatusEnum.READY) {
+                throw new IllegalStateException("At least one segment is not in READY state");
+            }
+        }
+
+        checkAndGetHbaseUrl();
+
+        hdfsFS = HadoopUtil.getWorkingFileSystem();
+
+        operations = new ArrayList<Opt>();
+
+        copyFilesInMetaStore(cube, overwriteIfExists);
+        renameFoldersInHdfs(cube);
+        addCubeAndModelIntoProject(cube, cubeName, projectName);
+        if (Boolean.parseBoolean(copyAcl) == true) {
+            copyACL(cube, projectName);
+        }
+
+        if (Boolean.parseBoolean(purgeAndDisable) == true) {
+            purgeAndDisable(cubeName); // this should be the last action
+        }
+
+        if (realExecute.equalsIgnoreCase("true")) {
+            doOpts();
+            updateMeta(dstConfig, projectName, cubeName, cube.getModel());
+            updateOnlineVersion(cube.getDescriptor());
+        } else {
+            showOpts();
+        }
+    }
+
+    public void moveCube(String srcCfgUri, String dstCfgUri, String cubeName, String projectName, String copyAcl, String purgeAndDisable, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
+        moveCube(KylinConfig.createInstanceFromUri(srcCfgUri), KylinConfig.createInstanceFromUri(dstCfgUri), cubeName, projectName, copyAcl, purgeAndDisable, overwriteIfExists, realExecute);
+    }
+
+    private void updateOnlineVersion(CubeDesc cubeDesc) throws IOException {
+        cubeDesc.addOnlineVersion();
+        CubeDescManager cubeDescManager = CubeDescManager.getInstance(srcConfig);
+        cubeDescManager.updateCubeDesc(cubeDesc);
+    }
+
+    protected void checkAndGetHbaseUrl() {
+        StorageURL srcMetadataUrl = srcConfig.getMetadataUrl();
+        StorageURL dstMetadataUrl = dstConfig.getMetadataUrl();
+
+        logger.info("src metadata url is " + srcMetadataUrl);
+        logger.info("dst metadata url is " + dstMetadataUrl);
+
+        if (!"hbase".equals(srcMetadataUrl.getScheme()) || !"hbase".equals(dstMetadataUrl.getScheme()))
+            throw new IllegalStateException("Both metadata urls should be hbase metadata url");
+    }
+
+    protected void renameFoldersInHdfs(CubeInstance cube) throws IOException {
+        for (CubeSegment segment : cube.getSegments()) {
+
+            String jobUuid = segment.getLastBuildJobID();
+            String src = JobBuilderSupport.getJobWorkingDir(srcConfig.getHdfsWorkingDirectory(), jobUuid);
+            String tgt = JobBuilderSupport.getJobWorkingDir(dstConfig.getHdfsWorkingDirectory(), jobUuid);
+
+            operations.add(new Opt(OptType.RENAME_FOLDER_IN_HDFS, new Object[] { src, tgt }));
+        }
+
+    }
+
+    private void copyACL(CubeInstance cube, String projectName) {
+        operations.add(new Opt(OptType.COPY_ACL, new Object[] { cube.getUuid(), cube.getDescriptor().getModel().getUuid(), projectName }));
+    }
+
+    private void copyFilesInMetaStore(CubeInstance cube, String overwriteIfExists) throws IOException {
+
+        List<String> metaItems = new ArrayList<String>();
+        Set<String> dictAndSnapshot = new HashSet<String>();
+        listCubeRelatedResources(cube, metaItems, dictAndSnapshot);
+
+        if (dstStore.exists(cube.getResourcePath()) && !overwriteIfExists.equalsIgnoreCase("true"))
+            throw new IllegalStateException("The cube named " + cube.getName() + " already exists on target metadata store. Use overwriteIfExists to overwrite it");
+
+        for (String item : metaItems) {
+            operations.add(new Opt(OptType.COPY_FILE_IN_META, new Object[] { item }));
+        }
+
+        for (String item : dictAndSnapshot) {
+            operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
+        }
+    }
+
+    private void addCubeAndModelIntoProject(CubeInstance srcCube, String cubeName, String projectName) throws IOException {
+        String projectResPath = ProjectInstance.concatResourcePath(projectName);
+        if (!dstStore.exists(projectResPath))
+            throw new IllegalStateException("The target project " + projectName + " does not exist");
+
+        operations.add(new Opt(OptType.ADD_INTO_PROJECT, new Object[] { srcCube, cubeName, projectName }));
+    }
+
+    private void purgeAndDisable(String cubeName) throws IOException {
+        operations.add(new Opt(OptType.PURGE_AND_DISABLE, new Object[] { cubeName }));
+    }
+
+    protected void listCubeRelatedResources(CubeInstance cube, List<String> metaResource, Set<String> dictAndSnapshot) throws IOException {
+
+        CubeDesc cubeDesc = cube.getDescriptor();
+        String prj = cubeDesc.getProject();
+        metaResource.add(cube.getResourcePath());
+        metaResource.add(cubeDesc.getResourcePath());
+        metaResource.add(DataModelDesc.concatResourcePath(cubeDesc.getModelName()));
+
+        for (TableRef tableRef : cubeDesc.getModel().getAllTables()) {
+            metaResource.add(TableDesc.concatResourcePath(tableRef.getTableIdentity(), prj));
+            metaResource.add(TableExtDesc.concatResourcePath(tableRef.getTableIdentity(), prj));
+        }
+
+        for (CubeSegment segment : cube.getSegments()) {
+            metaResource.add(segment.getStatisticsResourcePath());
+            dictAndSnapshot.addAll(segment.getSnapshotPaths());
+            dictAndSnapshot.addAll(segment.getDictionaryPaths());
+        }
+    }
+
+    protected enum OptType {
+        COPY_FILE_IN_META, COPY_DICT_OR_SNAPSHOT, RENAME_FOLDER_IN_HDFS, ADD_INTO_PROJECT, COPY_ACL, PURGE_AND_DISABLE
+    }
+
+    protected void addOpt(OptType type, Object[] params) {
+        operations.add(new Opt(type, params));
+    }
+
+    private class Opt {
+        private OptType type;
+        private Object[] params;
+
+        private Opt(OptType type, Object[] params) {
+            this.type = type;
+            this.params = params;
+        }
+
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append(type).append(":");
+            for (Object s : params)
+                sb.append(s).append(", ");
+            return sb.toString();
+        }
+
+    }
+
+    private void showOpts() {
+        for (int i = 0; i < operations.size(); ++i) {
+            showOpt(operations.get(i));
+        }
+    }
+
+    private void showOpt(Opt opt) {
+        logger.info("Operation: " + opt.toString());
+    }
+
+    private void doOpts() throws IOException, InterruptedException {
+        int index = 0;
+        try {
+            for (; index < operations.size(); ++index) {
+                logger.info("Operation index :" + index);
+                doOpt(operations.get(index));
+            }
+        } catch (Exception e) {
+            logger.error("error met", e);
+            logger.info("Try undoing previous changes");
+            // undo:
+            for (int i = index; i >= 0; --i) {
+                try {
+                    undo(operations.get(i));
+                } catch (Exception ee) {
+                    logger.error("error met ", e);
+                    logger.info("Continue undoing...");
+                }
+            }
+
+            throw new RuntimeException("Cube moving failed");
+        }
+    }
+
+    @SuppressWarnings("checkstyle:methodlength")
+    private void doOpt(Opt opt) throws IOException, InterruptedException {
+        logger.info("Executing operation: " + opt.toString());
+
+        switch (opt.type) {
+        case COPY_FILE_IN_META: {
+            String item = (String) opt.params[0];
+            RawResource res = srcStore.getResource(item);
+            dstStore.putResource(item, res.inputStream, res.timestamp);
+            res.inputStream.close();
+            logger.info("Item " + item + " is copied");
+            break;
+        }
+        case COPY_DICT_OR_SNAPSHOT: {
+            String item = (String) opt.params[0];
+
+            if (item.toLowerCase(Locale.ROOT).endsWith(".dict")) {
+                DictionaryManager dstDictMgr = DictionaryManager.getInstance(dstConfig);
+                DictionaryManager srcDicMgr = DictionaryManager.getInstance(srcConfig);
+                DictionaryInfo dictSrc = srcDicMgr.getDictionaryInfo(item);
+
+                long ts = dictSrc.getLastModified();
+                dictSrc.setLastModified(0);//to avoid resource store write conflict
+                Dictionary dictObj = dictSrc.getDictionaryObject().copyToAnotherMeta(srcConfig, dstConfig);
+                DictionaryInfo dictSaved = dstDictMgr.trySaveNewDict(dictObj, dictSrc);
+                dictSrc.setLastModified(ts);
+
+                if (dictSaved == dictSrc) {
+                    //no dup found, already saved to dest
+                    logger.info("Item " + item + " is copied");
+                } else {
+                    //dictSrc is rejected because of duplication
+                    //modify cube's dictionary path
+                    String cubeName = (String) opt.params[1];
+                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                    for (CubeSegment segment : cube.getSegments()) {
+                        for (Map.Entry<String, String> entry : segment.getDictionaries().entrySet()) {
+                            if (entry.getValue().equalsIgnoreCase(item)) {
+                                entry.setValue(dictSaved.getResourcePath());
+                            }
+                        }
+                    }
+                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
+                    logger.info("Item " + item + " is dup, instead " + dictSaved.getResourcePath() + " is reused");
+                }
+
+            } else if (item.toLowerCase(Locale.ROOT).endsWith(".snapshot")) {
+                SnapshotManager dstSnapMgr = SnapshotManager.getInstance(dstConfig);
+                SnapshotManager srcSnapMgr = SnapshotManager.getInstance(srcConfig);
+                SnapshotTable snapSrc = srcSnapMgr.getSnapshotTable(item);
+
+                long ts = snapSrc.getLastModified();
+                snapSrc.setLastModified(0);
+                SnapshotTable snapSaved = dstSnapMgr.trySaveNewSnapshot(snapSrc);
+                snapSrc.setLastModified(ts);
+
+                if (snapSaved == snapSrc) {
+                    //no dup found, already saved to dest
+                    logger.info("Item " + item + " is copied");
+
+                } else {
+                    String cubeName = (String) opt.params[1];
+                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+                    for (CubeSegment segment : cube.getSegments()) {
+                        for (Map.Entry<String, String> entry : segment.getSnapshots().entrySet()) {
+                            if (entry.getValue().equalsIgnoreCase(item)) {
+                                entry.setValue(snapSaved.getResourcePath());
+                            }
+                        }
+                    }
+                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
+                    logger.info("Item " + item + " is dup, instead " + snapSaved.getResourcePath() + " is reused");
+
+                }
+
+            } else {
+                logger.error("unknown item found: " + item);
+                logger.info("ignore it");
+            }
+
+            break;
+        }
+        case RENAME_FOLDER_IN_HDFS: {
+            String srcPath = (String) opt.params[0];
+            String dstPath = (String) opt.params[1];
+            renameHDFSPath(srcPath, dstPath);
+            logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
+            break;
+        }
+        case ADD_INTO_PROJECT: {
+            CubeInstance srcCube = (CubeInstance) opt.params[0];
+            String cubeName = (String) opt.params[1];
+            String projectName = (String) opt.params[2];
+            String modelName = srcCube.getDescriptor().getModelName();
+
+            String projectResPath = ProjectInstance.concatResourcePath(projectName);
+            Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
+            ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
+
+            for (TableRef tableRef : srcCube.getModel().getAllTables()) {
+                project.addTable(tableRef.getTableIdentity());
+            }
+
+            project.addModel(modelName);
+            project.removeRealization(RealizationType.CUBE, cubeName);
+            project.addRealizationEntry(RealizationType.CUBE, cubeName);
+
+            dstStore.putResource(projectResPath, project, projectSerializer);
+            logger.info("Project instance for " + projectName + " is corrected");
+            break;
+        }
+        case COPY_ACL: {
+            String cubeId = (String) opt.params[0];
+            String modelId = (String) opt.params[1];
+            String projectName = (String) opt.params[2];
+            String projectResPath = ProjectInstance.concatResourcePath(projectName);
+            Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
+            ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
+            String projUUID = project.getUuid();
+            Table srcAclHtable = null;
+            Table destAclHtable = null;
+            try {
+                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+
+                // cube acl
+                Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
+                if (result.listCells() != null) {
+                    for (Cell cell : result.listCells()) {
+                        byte[] family = CellUtil.cloneFamily(cell);
+                        byte[] column = CellUtil.cloneQualifier(cell);
+                        byte[] value = CellUtil.cloneValue(cell);
+
+                        // use the target project uuid as the parent
+                        if (Bytes.toString(family).equals(ACL_INFO_FAMILY) && Bytes.toString(column).equals(ACL_INFO_FAMILY_PARENT_COLUMN)) {
+                            String valueString = "{\"id\":\"" + projUUID + "\",\"type\":\"org.apache.kylin.metadata.project.ProjectInstance\"}";
+                            value = Bytes.toBytes(valueString);
+                        }
+                        Put put = new Put(Bytes.toBytes(cubeId));
+                        put.add(family, column, value);
+                        destAclHtable.put(put);
+                    }
+                }
+            } finally {
+                IOUtils.closeQuietly(srcAclHtable);
+                IOUtils.closeQuietly(destAclHtable);
+            }
+            break;
+        }
+        case PURGE_AND_DISABLE: {
+            String cubeName = (String) opt.params[0];
+            String cubeResPath = CubeInstance.concatResourcePath(cubeName);
+            Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
+            CubeInstance cube = srcStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
+            cube.getSegments().clear();
+            cube.setStatus(RealizationStatusEnum.DISABLED);
+            srcStore.putResource(cubeResPath, cube, cubeSerializer);
+            logger.info("Cube " + cubeName + " is purged and disabled in " + srcConfig.getMetadataUrl());
+
+            break;
+        }
+        default: {
+            //do nothing
+            break;
+        }
+        }
+    }
+
+    private void undo(Opt opt) throws IOException, InterruptedException {
+        logger.info("Undo operation: " + opt.toString());
+
+        switch (opt.type) {
+        case COPY_FILE_IN_META: {
+            // no harm
+            logger.info("Undo for COPY_FILE_IN_META is ignored");
+            break;
+        }
+        case COPY_DICT_OR_SNAPSHOT: {
+            // no harm
+            logger.info("Undo for COPY_DICT_OR_SNAPSHOT is ignored");
+            break;
+        }
+        case RENAME_FOLDER_IN_HDFS: {
+            String srcPath = (String) opt.params[1];
+            String dstPath = (String) opt.params[0];
+
+            if (hdfsFS.exists(new Path(srcPath)) && !hdfsFS.exists(new Path(dstPath))) {
+                renameHDFSPath(srcPath, dstPath);
+                logger.info("HDFS Folder renamed from " + srcPath + " to " + dstPath);
+            }
+            break;
+        }
+        case ADD_INTO_PROJECT: {
+            logger.info("Undo for ADD_INTO_PROJECT is ignored");
+            break;
+        }
+        case COPY_ACL: {
+            String cubeId = (String) opt.params[0];
+            String modelId = (String) opt.params[1];
+            Table destAclHtable = null;
+            try {
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+
+                destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
+                destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
+            } finally {
+                IOUtils.closeQuietly(destAclHtable);
+            }
+            break;
+        }
+        case PURGE_AND_DISABLE: {
+            logger.info("Undo for PURGE_AND_DISABLE is not supported");
+            break;
+        }
+        default: {
+            //do nothing
+            break;
+        }
+        }
+    }
+
+    private void updateMeta(KylinConfig config, String projectName, String cubeName, DataModelDesc model) {
+        String[] nodes = config.getRestServers();
+        Map<String, String> tableToProjects = new HashMap<>();
+        for (TableRef tableRef : model.getAllTables()) {
+            tableToProjects.put(tableRef.getTableIdentity(), tableRef.getTableDesc().getProject());
+        }
+
+        for (String node : nodes) {
+            RestClient restClient = new RestClient(node);
+            try {
+                logger.info("update meta cache for " + node);
+                restClient.clearCacheForCubeMigration(cubeName, projectName, model.getName(), tableToProjects);
+            } catch (IOException e) {
+                logger.error(e.getMessage());
+            }
+        }
+    }
+
+    private void renameHDFSPath(String srcPath, String dstPath) throws IOException, InterruptedException {
+        int nRetry = 0;
+        int sleepTime = 5000;
+        while (!hdfsFS.rename(new Path(srcPath), new Path(dstPath))) {
+            ++nRetry;
+            if (nRetry > 3) {
+                throw new InterruptedException("Cannot rename folder " + srcPath + " to folder " + dstPath);
+            } else {
+                Thread.sleep(sleepTime * nRetry * nRetry);
+            }
+        }
+    }
+}
diff --git a/webapp/app/js/model/cubeConfig.js b/webapp/app/js/model/cubeConfig.js
index a83d4c9a66..d10e1f13d6 100644
--- a/webapp/app/js/model/cubeConfig.js
+++ b/webapp/app/js/model/cubeConfig.js
@@ -27,6 +27,10 @@ KylinApp.constant('cubeConfig', {
     {name:'MapReduce',value: 2},
     {name:'Spark',value: 4}
   ],
+  storageType:[
+    {name:'HBase',value: 2},
+    {name:'Druid',value: 5}
+  ],
   joinTypes: [
     {name: 'Left', value: 'left'},
     {name: 'Inner', value: 'inner'}
@@ -200,4 +204,4 @@ KylinApp.constant('cubeConfig', {
       'left': '-12px'
     }
   }
-});
\ No newline at end of file
+});
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index 89229d039c..dcbc0f85d3 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -385,6 +385,27 @@ <h3 style="margin-left:42px;margin-bottom:30px;">Cube Engine  <i kylinpopover pl
             </div>
           </div>
         </div>
+
+        <!--Cube Storage-->
+        <div class="form-group large-popover" style="margin-bottom:30px;">
+          <h3 style="margin-left:42px;margin-bottom:30px;">Cube Storage  <i kylinpopover placement="right" title="Cube Storage" template="CubeStorageTip.html" class="fa fa-info-circle"></i></h3>
+          <div class="row" style="margin-left:42px">
+            <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Storage Type :</b></label>
+            <div class="col-xs-12 col-sm-6">
+              <select style="width: 100%" chosen
+                      ng-model="cubeMetaFrame.storage_type"
+                      ng-if="state.mode=='edit'"
+                      ng-disabled="instance.segments.length>0"
+                      ng-options="et.value as et.name for et in cubeConfig.storageType">
+                <option value="">--Select Storage Type--</option>
+              </select>
+              <span ng-if="state.mode=='view'&&cubeMetaFrame.storage_type==2">HBase</span>
+              <span ng-if="state.mode=='view'&&cubeMetaFrame.storage_type==5">Druid</span>
+            </div>
+          </div>
+        </div>
+
+
         <div class="form-group large-popover">
           <h3 style="margin-left:42px">Advanced Dictionaries  <i kylinpopover placement="right" title="Advanced Dictionaries" template="AdvancedDictionariesTip.html" class="fa fa-info-circle"></i></h3>
           <div style="margin-left:42px">
@@ -748,3 +769,11 @@ <h4>
     <h4>Advance snapshot design for global lookup table and provide different storage type.</h4>
   </div>
 </script>
+
+<script type="text/ng-template" id="CubeStorageTip.html" >
+  <div>
+    <h4>
+      Select cube storage.
+    </h4>
+  </div>
+</script>


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services