You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/06/25 23:34:23 UTC

[11/13] kylin git commit: KYLIN-1823 split code from kylin-server into kylin-server-base

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingController.java
new file mode 100644
index 0000000..f3374c3
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/StreamingController.java
@@ -0,0 +1,297 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.streaming.StreamingConfig;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.rest.exception.BadRequestException;
+import org.apache.kylin.rest.exception.ForbiddenException;
+import org.apache.kylin.rest.exception.InternalErrorException;
+import org.apache.kylin.rest.exception.NotFoundException;
+import org.apache.kylin.rest.request.StreamingRequest;
+import org.apache.kylin.rest.service.CubeService;
+import org.apache.kylin.rest.service.KafkaConfigService;
+import org.apache.kylin.rest.service.StreamingService;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.AccessDeniedException;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+
+/**
+ * StreamingController is defined as Restful API entrance for UI.
+ *
+ * @author jiazhong
+ */
+@Controller
+@RequestMapping(value = "/streaming")
+public class StreamingController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(StreamingController.class);
+
+    @Autowired
+    private StreamingService streamingService;
+    @Autowired
+    private KafkaConfigService kafkaConfigService;
+    @Autowired
+    private CubeService cubeMgmtService;
+
+    @RequestMapping(value = "/getConfig", method = { RequestMethod.GET })
+    @ResponseBody
+    public List<StreamingConfig> getStreamings(@RequestParam(value = "table", required = false) String table, @RequestParam(value = "limit", required = false) Integer limit, @RequestParam(value = "offset", required = false) Integer offset) {
+        try {
+            return streamingService.getStreamingConfigs(table, limit, offset);
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage());
+        }
+    }
+
+    @RequestMapping(value = "/getKfkConfig", method = { RequestMethod.GET })
+    @ResponseBody
+    public List<KafkaConfig> getKafkaConfigs(@RequestParam(value = "kafkaConfigName", required = false) String kafkaConfigName, @RequestParam(value = "limit", required = false) Integer limit, @RequestParam(value = "offset", required = false) Integer offset) {
+        try {
+            return kafkaConfigService.getKafkaConfigs(kafkaConfigName, limit, offset);
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage());
+        }
+    }
+
+    /**
+     *
+     * create Streaming Schema
+     * @throws java.io.IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.POST })
+    @ResponseBody
+    public StreamingRequest saveStreamingConfig(@RequestBody StreamingRequest streamingRequest) {
+
+        String project = streamingRequest.getProject();
+        TableDesc tableDesc = deserializeTableDesc(streamingRequest);
+        StreamingConfig streamingConfig = deserializeSchemalDesc(streamingRequest);
+        KafkaConfig kafkaConfig = deserializeKafkaSchemalDesc(streamingRequest);
+        boolean saveStreamingSuccess = false, saveKafkaSuccess = false;
+
+        try {
+            tableDesc.setUuid(UUID.randomUUID().toString());
+            MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+            metaMgr.saveSourceTable(tableDesc);
+            cubeMgmtService.syncTableToProject(new String[] { tableDesc.getIdentity() }, project);
+        } catch (IOException e) {
+            throw new BadRequestException("Failed to add streaming table.");
+        }
+
+        streamingConfig.setName(tableDesc.getIdentity());
+        kafkaConfig.setName(tableDesc.getIdentity());
+        try {
+            if (StringUtils.isEmpty(streamingConfig.getName())) {
+                logger.info("StreamingConfig should not be empty.");
+                throw new BadRequestException("StremingConfig name should not be empty.");
+            }
+            try {
+                streamingConfig.setUuid(UUID.randomUUID().toString());
+                streamingService.createStreamingConfig(streamingConfig);
+                saveStreamingSuccess = true;
+            } catch (IOException e) {
+                logger.error("Failed to save StreamingConfig:" + e.getLocalizedMessage(), e);
+                throw new InternalErrorException("Failed to save StreamingConfig: " + e.getLocalizedMessage());
+            }
+            try {
+                kafkaConfig.setUuid(UUID.randomUUID().toString());
+                kafkaConfigService.createKafkaConfig(kafkaConfig);
+                saveKafkaSuccess = true;
+            } catch (IOException e) {
+                try {
+                    streamingService.dropStreamingConfig(streamingConfig);
+                } catch (IOException e1) {
+                    throw new InternalErrorException("StreamingConfig is created, but failed to create KafkaConfig: " + e.getLocalizedMessage());
+                }
+                logger.error("Failed to save KafkaConfig:" + e.getLocalizedMessage(), e);
+                throw new InternalErrorException("Failed to save KafkaConfig: " + e.getLocalizedMessage());
+            }
+        } finally {
+            if (saveKafkaSuccess == false || saveStreamingSuccess == false) {
+
+                if (saveStreamingSuccess == true) {
+                    StreamingConfig sConfig = streamingService.getStreamingManager().getStreamingConfig(streamingConfig.getName());
+                    try {
+                        streamingService.dropStreamingConfig(sConfig);
+                    } catch (IOException e) {
+                        throw new InternalErrorException("Action failed and failed to rollback the created streaming config: " + e.getLocalizedMessage());
+                    }
+                }
+                if (saveKafkaSuccess == true) {
+                    try {
+                        KafkaConfig kConfig = kafkaConfigService.getKafkaConfig(kafkaConfig.getName());
+                        kafkaConfigService.dropKafkaConfig(kConfig);
+                    } catch (IOException e) {
+                        throw new InternalErrorException("Action failed and failed to rollback the created kafka config: " + e.getLocalizedMessage());
+                    }
+                }
+            }
+
+        }
+        streamingRequest.setSuccessful(true);
+        return streamingRequest;
+    }
+
+    @RequestMapping(value = "", method = { RequestMethod.PUT })
+    @ResponseBody
+    public StreamingRequest updateStreamingConfig(@RequestBody StreamingRequest streamingRequest) throws JsonProcessingException {
+        StreamingConfig streamingConfig = deserializeSchemalDesc(streamingRequest);
+        KafkaConfig kafkaConfig = deserializeKafkaSchemalDesc(streamingRequest);
+
+        if (streamingConfig == null) {
+            return streamingRequest;
+        }
+        try {
+            streamingConfig = streamingService.updateStreamingConfig(streamingConfig);
+        } catch (AccessDeniedException accessDeniedException) {
+            throw new ForbiddenException("You don't have right to update this StreamingConfig.");
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage());
+        }
+        try {
+            kafkaConfig = kafkaConfigService.updateKafkaConfig(kafkaConfig);
+        } catch (AccessDeniedException accessDeniedException) {
+            throw new ForbiddenException("You don't have right to update this KafkaConfig.");
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage());
+        }
+
+        streamingRequest.setSuccessful(true);
+
+        return streamingRequest;
+    }
+
+    @RequestMapping(value = "/{configName}", method = { RequestMethod.DELETE })
+    @ResponseBody
+    public void deleteConfig(@PathVariable String configName) throws IOException {
+        StreamingConfig config = streamingService.getStreamingManager().getStreamingConfig(configName);
+        KafkaConfig kafkaConfig = kafkaConfigService.getKafkaConfig(configName);
+        if (null == config) {
+            throw new NotFoundException("StreamingConfig with name " + configName + " not found..");
+        }
+        try {
+            streamingService.dropStreamingConfig(config);
+            kafkaConfigService.dropKafkaConfig(kafkaConfig);
+        } catch (Exception e) {
+            logger.error(e.getLocalizedMessage(), e);
+            throw new InternalErrorException("Failed to delete StreamingConfig. " + " Caused by: " + e.getMessage(), e);
+        }
+    }
+
+    private TableDesc deserializeTableDesc(StreamingRequest streamingRequest) {
+        TableDesc desc = null;
+        try {
+            logger.debug("Saving TableDesc " + streamingRequest.getTableData());
+            desc = JsonUtil.readValue(streamingRequest.getTableData(), TableDesc.class);
+        } catch (JsonParseException e) {
+            logger.error("The TableDesc definition is invalid.", e);
+            updateRequest(streamingRequest, false, e.getMessage());
+        } catch (JsonMappingException e) {
+            logger.error("The data TableDesc definition is invalid.", e);
+            updateRequest(streamingRequest, false, e.getMessage());
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e);
+        }
+
+        String[] dbTable = HadoopUtil.parseHiveTableName(desc.getName());
+        desc.setName(dbTable[1]);
+        desc.setDatabase(dbTable[0]);
+        desc.getIdentity();
+        return desc;
+    }
+
+    private StreamingConfig deserializeSchemalDesc(StreamingRequest streamingRequest) {
+        StreamingConfig desc = null;
+        try {
+            logger.debug("Saving StreamingConfig " + streamingRequest.getStreamingConfig());
+            desc = JsonUtil.readValue(streamingRequest.getStreamingConfig(), StreamingConfig.class);
+        } catch (JsonParseException e) {
+            logger.error("The StreamingConfig definition is invalid.", e);
+            updateRequest(streamingRequest, false, e.getMessage());
+        } catch (JsonMappingException e) {
+            logger.error("The data StreamingConfig definition is invalid.", e);
+            updateRequest(streamingRequest, false, e.getMessage());
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e);
+        }
+        return desc;
+    }
+
+    private KafkaConfig deserializeKafkaSchemalDesc(StreamingRequest streamingRequest) {
+        KafkaConfig desc = null;
+        try {
+            logger.debug("Saving KafkaConfig " + streamingRequest.getKafkaConfig());
+            desc = JsonUtil.readValue(streamingRequest.getKafkaConfig(), KafkaConfig.class);
+        } catch (JsonParseException e) {
+            logger.error("The KafkaConfig definition is invalid.", e);
+            updateRequest(streamingRequest, false, e.getMessage());
+        } catch (JsonMappingException e) {
+            logger.error("The data KafkaConfig definition is invalid.", e);
+            updateRequest(streamingRequest, false, e.getMessage());
+        } catch (IOException e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e);
+        }
+        return desc;
+    }
+
+    private void updateRequest(StreamingRequest request, boolean success, String message) {
+        request.setSuccessful(success);
+        request.setMessage(message);
+    }
+
+    public void setStreamingService(StreamingService streamingService) {
+        this.streamingService = streamingService;
+    }
+
+    public void setKafkaConfigService(KafkaConfigService kafkaConfigService) {
+        this.kafkaConfigService = kafkaConfigService;
+    }
+
+    public void setCubeService(CubeService cubeService) {
+        this.cubeMgmtService = cubeService;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
new file mode 100644
index 0000000..1c3377b
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -0,0 +1,350 @@
+/*
+ * 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.rest.controller;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.streaming.StreamingConfig;
+import org.apache.kylin.metadata.MetadataConstants;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.rest.exception.InternalErrorException;
+import org.apache.kylin.rest.request.CardinalityRequest;
+import org.apache.kylin.rest.request.HiveTableRequest;
+import org.apache.kylin.rest.request.StreamingRequest;
+import org.apache.kylin.rest.response.TableDescResponse;
+import org.apache.kylin.rest.service.CubeService;
+import org.apache.kylin.rest.service.KafkaConfigService;
+import org.apache.kylin.rest.service.ModelService;
+import org.apache.kylin.rest.service.ProjectService;
+import org.apache.kylin.rest.service.StreamingService;
+import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+import com.google.common.collect.Sets;
+
+/**
+ * @author xduo
+ */
+@Controller
+@RequestMapping(value = "/tables")
+public class TableController extends BasicController {
+    private static final Logger logger = LoggerFactory.getLogger(TableController.class);
+
+    @Autowired
+    private CubeService cubeMgmtService;
+    @Autowired
+    private ProjectService projectService;
+    @Autowired
+    private StreamingService streamingService;
+    @Autowired
+    private KafkaConfigService kafkaConfigService;
+    @Autowired
+    private ModelService modelService;
+
+    /**
+     * Get available table list of the input database
+     *
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "", method = { RequestMethod.GET })
+    @ResponseBody
+    public List<TableDesc> getHiveTables(@RequestParam(value = "ext", required = false) boolean withExt, @RequestParam(value = "project", required = false) String project) {
+        long start = System.currentTimeMillis();
+        List<TableDesc> tables = null;
+        try {
+            tables = cubeMgmtService.getProjectManager().listDefinedTables(project);
+        } catch (Exception e) {
+            logger.error("Failed to deal with the request.", e);
+            throw new InternalErrorException(e.getLocalizedMessage());
+        }
+
+        if (withExt) {
+            tables = cloneTableDesc(tables);
+        }
+        long end = System.currentTimeMillis();
+        logger.info("Return all table metadata in " + (end - start) + " seconds");
+
+        return tables;
+    }
+
+    /**
+     * Get available table list of the input database
+     *
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{tableName:.+}", method = { RequestMethod.GET })
+    @ResponseBody
+    public TableDesc getHiveTable(@PathVariable String tableName) {
+        return cubeMgmtService.getMetadataManager().getTableDesc(tableName);
+    }
+
+    /**
+     * Get available table list of the input database
+     *
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{tableName}/exd-map", method = { RequestMethod.GET })
+    @ResponseBody
+    public Map<String, String> getHiveTableExd(@PathVariable String tableName) {
+        Map<String, String> tableExd = cubeMgmtService.getMetadataManager().getTableDescExd(tableName);
+        return tableExd;
+    }
+
+    @RequestMapping(value = "/reload", method = { RequestMethod.PUT })
+    @ResponseBody
+    public String reloadSourceTable() {
+        cubeMgmtService.getMetadataManager().reload();
+        return "ok";
+    }
+
+    @RequestMapping(value = "/{tables}/{project}", method = { RequestMethod.POST })
+    @ResponseBody
+    public Map<String, String[]> loadHiveTable(@PathVariable String tables, @PathVariable String project, @RequestBody HiveTableRequest request) throws IOException {
+        String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
+        String[] loaded = cubeMgmtService.reloadHiveTable(tables);
+        if (request.isCalculate()) {
+            cubeMgmtService.calculateCardinalityIfNotPresent(loaded, submitter);
+        }
+        cubeMgmtService.syncTableToProject(loaded, project);
+        Map<String, String[]> result = new HashMap<String, String[]>();
+        result.put("result.loaded", loaded);
+        result.put("result.unloaded", new String[] {});
+        return result;
+    }
+
+    @RequestMapping(value = "/{tables}/{project}", method = { RequestMethod.DELETE })
+    @ResponseBody
+    public Map<String, String[]> unLoadHiveTables(@PathVariable String tables, @PathVariable String project) {
+        Set<String> unLoadSuccess = Sets.newHashSet();
+        Set<String> unLoadFail = Sets.newHashSet();
+        Map<String, String[]> result = new HashMap<String, String[]>();
+        for (String tableName : tables.split(",")) {
+            if (unLoadHiveTable(tableName, project)) {
+                unLoadSuccess.add(tableName);
+            } else {
+                unLoadFail.add(tableName);
+            }
+        }
+        result.put("result.unload.success", (String[]) unLoadSuccess.toArray(new String[unLoadSuccess.size()]));
+        result.put("result.unload.fail", (String[]) unLoadFail.toArray(new String[unLoadFail.size()]));
+        return result;
+    }
+
+    /**
+     * table may referenced by several projects, and kylin only keep one copy of meta for each table,
+     * that's why we have two if statement here.
+     * @param tableName
+     * @param project
+     * @return
+     */
+    private boolean unLoadHiveTable(String tableName, String project) {
+        boolean rtn = false;
+        int tableType = 0;
+
+        //remove streaming info
+        String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
+        tableName = dbTableName[0] + "." + dbTableName[1];
+        TableDesc desc = cubeMgmtService.getMetadataManager().getTableDesc(tableName);
+        tableType = desc.getSourceType();
+
+        try {
+            if (!modelService.isTableInModel(tableName, project)) {
+                cubeMgmtService.removeTableFromProject(tableName, project);
+                rtn = true;
+            }
+        } catch (IOException e) {
+            logger.error(e.getMessage(), e);
+        }
+        if (!projectService.isTableInAnyProject(tableName) && !modelService.isTableInAnyModel(tableName)) {
+            try {
+                cubeMgmtService.unLoadHiveTable(tableName);
+                rtn = true;
+            } catch (IOException e) {
+                logger.error(e.getMessage(), e);
+                rtn = false;
+            }
+        }
+
+        if (tableType == 1 && !projectService.isTableInAnyProject(tableName) && !modelService.isTableInAnyModel(tableName)) {
+            StreamingConfig config = null;
+            KafkaConfig kafkaConfig = null;
+            try {
+                config = streamingService.getStreamingManager().getStreamingConfig(tableName);
+                kafkaConfig = kafkaConfigService.getKafkaConfig(tableName);
+                streamingService.dropStreamingConfig(config);
+                kafkaConfigService.dropKafkaConfig(kafkaConfig);
+                rtn = true;
+            } catch (Exception e) {
+                rtn = false;
+                logger.error(e.getLocalizedMessage(), e);
+            }
+        }
+        return rtn;
+    }
+
+    @RequestMapping(value = "/addStreamingSrc", method = { RequestMethod.POST })
+    @ResponseBody
+    public Map<String, String> addStreamingTable(@RequestBody StreamingRequest request) throws IOException {
+        Map<String, String> result = new HashMap<String, String>();
+        String project = request.getProject();
+        TableDesc desc = JsonUtil.readValue(request.getTableData(), TableDesc.class);
+        desc.setUuid(UUID.randomUUID().toString());
+        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+        metaMgr.saveSourceTable(desc);
+        cubeMgmtService.syncTableToProject(new String[] { desc.getName() }, project);
+        result.put("success", "true");
+        return result;
+    }
+
+    /**
+     * Regenerate table cardinality
+     *
+     * @return Table metadata array
+     * @throws IOException
+     */
+    @RequestMapping(value = "/{tableNames}/cardinality", method = { RequestMethod.PUT })
+    @ResponseBody
+    public CardinalityRequest generateCardinality(@PathVariable String tableNames, @RequestBody CardinalityRequest request) {
+        String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
+        String[] tables = tableNames.split(",");
+        for (String table : tables) {
+            cubeMgmtService.calculateCardinality(table.trim().toUpperCase(), submitter);
+        }
+        return request;
+    }
+
+    /**
+     * @param tables
+     * @return
+     */
+    private List<TableDesc> cloneTableDesc(List<TableDesc> tables) {
+        if (null == tables) {
+            return Collections.emptyList();
+        }
+
+        List<TableDesc> descs = new ArrayList<TableDesc>();
+        Iterator<TableDesc> it = tables.iterator();
+        while (it.hasNext()) {
+            TableDesc table = it.next();
+            Map<String, String> exd = cubeMgmtService.getMetadataManager().getTableDescExd(table.getIdentity());
+            if (exd == null) {
+                descs.add(table);
+            } else {
+                // Clone TableDesc
+                TableDescResponse rtableDesc = new TableDescResponse(table);
+                rtableDesc.setDescExd(exd);
+                if (exd.containsKey(MetadataConstants.TABLE_EXD_CARDINALITY)) {
+                    Map<String, Long> cardinality = new HashMap<String, Long>();
+                    String scard = exd.get(MetadataConstants.TABLE_EXD_CARDINALITY);
+                    if (!StringUtils.isEmpty(scard)) {
+                        String[] cards = StringUtils.split(scard, ",");
+                        ColumnDesc[] cdescs = rtableDesc.getColumns();
+                        for (int i = 0; i < cdescs.length; i++) {
+                            ColumnDesc columnDesc = cdescs[i];
+                            if (cards.length > i) {
+                                cardinality.put(columnDesc.getName(), Long.parseLong(cards[i]));
+                            } else {
+                                logger.error("The result cardinality is not identical with hive table metadata, cardinaly : " + scard + " column array length: " + cdescs.length);
+                                break;
+                            }
+                        }
+                        rtableDesc.setCardinality(cardinality);
+                    }
+                }
+                descs.add(rtableDesc);
+            }
+        }
+        return descs;
+    }
+
+    /**
+     * Show all databases in Hive
+     *
+     * @return Hive databases list
+     * @throws IOException
+     */
+    @RequestMapping(value = "/hive", method = { RequestMethod.GET })
+    @ResponseBody
+    private static List<String> showHiveDatabases() throws IOException {
+        HiveClient hiveClient = new HiveClient();
+        List<String> results = null;
+
+        try {
+            results = hiveClient.getHiveDbNames();
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new IOException(e);
+        }
+        return results;
+    }
+
+    /**
+     * Show all tables in a Hive database
+     *
+     * @return Hive table list
+     * @throws IOException
+     */
+    @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
+    @ResponseBody
+    private static List<String> showHiveTables(@PathVariable String database) throws IOException {
+        HiveClient hiveClient = new HiveClient();
+        List<String> results = null;
+
+        try {
+            results = hiveClient.getHiveTableNames(database);
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new IOException(e);
+        }
+        return results;
+    }
+
+    public void setCubeService(CubeService cubeService) {
+        this.cubeMgmtService = cubeService;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/controller/UserController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/UserController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/UserController.java
new file mode 100644
index 0000000..bccfed4
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/UserController.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rest.controller;
+
+import java.util.List;
+
+import org.apache.kylin.rest.service.UserService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.core.userdetails.UserDetails;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+
+/**
+ * Handle user authentication request to protected kylin rest resources by
+ * spring security.
+ * 
+ * @author xduo
+ * 
+ */
+@Controller
+@RequestMapping(value = "/user")
+public class UserController {
+
+    private static final Logger logger = LoggerFactory.getLogger(UserController.class);
+    @Autowired
+    UserService userService;
+
+    @RequestMapping(value = "/authentication", method = RequestMethod.POST, produces = "application/json")
+    public UserDetails authenticate() {
+        return authenticatedUser();
+    }
+
+    @RequestMapping(value = "/authentication", method = RequestMethod.GET, produces = "application/json")
+    public UserDetails authenticatedUser() {
+        Authentication authentication = SecurityContextHolder.getContext().getAuthentication();
+
+        if (authentication == null) {
+            logger.debug("authentication is null.");
+            return null;
+        }
+
+        if (authentication.getPrincipal() instanceof UserDetails) {
+            logger.debug("authentication.getPrincipal() is " + authentication.getPrincipal());
+            return (UserDetails) authentication.getPrincipal();
+        }
+
+        if (authentication.getDetails() instanceof UserDetails) {
+            logger.debug("authentication.getDetails() is " + authentication.getDetails());
+            return (UserDetails) authentication.getDetails();
+        }
+
+        return null;
+    }
+
+    @RequestMapping(value = "/authentication/authorities", method = RequestMethod.GET, produces = "application/json")
+    public List<String> getAuthorities() {
+        return userService.listUserAuthorities();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java b/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.java
new file mode 100644
index 0000000..42b671b
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/exception/BadRequestException.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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * @author xduo
+ *
+ */
+@ResponseStatus(value = HttpStatus.BAD_REQUEST)
+public class BadRequestException extends RuntimeException {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = -6798154278095441848L;
+
+    public BadRequestException(String s) {
+        super(s);
+    }
+
+    /**
+     * 
+     */
+    public BadRequestException() {
+        super();
+    }
+
+    /**
+     * @param arg0
+     * @param arg1
+     */
+    public BadRequestException(String arg0, Throwable arg1) {
+        super(arg0, arg1);
+    }
+
+    /**
+     * @param arg0
+     */
+    public BadRequestException(Throwable arg0) {
+        super(arg0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/exception/ForbiddenException.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/exception/ForbiddenException.java b/server-base/src/main/java/org/apache/kylin/rest/exception/ForbiddenException.java
new file mode 100644
index 0000000..6a2ed7c
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/exception/ForbiddenException.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * @author xduo
+ * 
+ */
+@ResponseStatus(value = HttpStatus.FORBIDDEN)
+public class ForbiddenException extends RuntimeException {
+
+    private static final long serialVersionUID = 2741885728370162194L;
+
+    public ForbiddenException() {
+        super();
+    }
+
+    public ForbiddenException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/exception/InternalErrorException.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/exception/InternalErrorException.java b/server-base/src/main/java/org/apache/kylin/rest/exception/InternalErrorException.java
new file mode 100644
index 0000000..6d7586f
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/exception/InternalErrorException.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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * Class to wrap backend exception
+ * 
+ * @author jianliu
+ * 
+ */
+@ResponseStatus(value = HttpStatus.INTERNAL_SERVER_ERROR)
+public class InternalErrorException extends RuntimeException {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = -6798154278095441848L;
+
+    public InternalErrorException(String s) {
+        super(s);
+    }
+
+    /**
+     * 
+     */
+    public InternalErrorException() {
+        super();
+    }
+
+    /**
+     * @param arg0
+     * @param arg1
+     */
+    public InternalErrorException(String arg0, Throwable arg1) {
+        super(arg0, arg1);
+    }
+
+    /**
+     * @param arg0
+     */
+    public InternalErrorException(Throwable arg0) {
+        super(arg0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/exception/NotFoundException.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/exception/NotFoundException.java b/server-base/src/main/java/org/apache/kylin/rest/exception/NotFoundException.java
new file mode 100644
index 0000000..0ed898c
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/exception/NotFoundException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rest.exception;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.web.bind.annotation.ResponseStatus;
+
+/**
+ * @author xduo
+ *
+ */
+@ResponseStatus(value = HttpStatus.NOT_FOUND)
+public class NotFoundException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+
+    public NotFoundException(String s) {
+        super(s);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/init/ClientInfoTask.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/init/ClientInfoTask.java b/server-base/src/main/java/org/apache/kylin/rest/init/ClientInfoTask.java
new file mode 100644
index 0000000..301a9df
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/init/ClientInfoTask.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.rest.init;
+
+import java.io.File;
+
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.KylinVersion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientInfoTask extends InitialTask {
+
+    private static final Logger logger = LoggerFactory.getLogger(ClientInfoTask.class);
+
+    @Override
+    public void execute() {
+        logger.info(getClientDetailInformation());
+    }
+
+    public static String getClientDetailInformation() {
+        StringBuilder buf = new StringBuilder();
+
+        buf.append("kylin.home: ").append(new File(KylinConfig.getKylinHome()).getAbsolutePath()).append("\n");
+        buf.append("kylin.version:").append(KylinVersion.getCurrentVersion()).append("\n");
+        buf.append("commit:").append(KylinVersion.getGitCommitInfo()).append("\n");
+        buf.append("os.name:").append(System.getProperty("os.name")).append("\n");
+        buf.append("os.arch:").append(System.getProperty("os.arch")).append("\n");
+        buf.append("os.version:").append(System.getProperty("os.version")).append("\n");
+        buf.append("java.version:").append(System.getProperty("java.version")).append("\n");
+        buf.append("java.vendor:").append(System.getProperty("java.vendor"));
+
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/init/InitialTask.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/init/InitialTask.java b/server-base/src/main/java/org/apache/kylin/rest/init/InitialTask.java
new file mode 100644
index 0000000..24691b1
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/init/InitialTask.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.rest.init;
+
+/**
+ * Created by dongli on 3/16/16.
+ */
+public abstract class InitialTask {
+    public abstract void execute();
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java b/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
new file mode 100644
index 0000000..8912968
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
@@ -0,0 +1,58 @@
+/*
+ * 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.rest.init;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.InitializingBean;
+
+/**
+ * Created by dongli on 3/16/16.
+ */
+public class InitialTaskManager implements InitializingBean {
+
+    private static final Logger logger = LoggerFactory.getLogger(InitialTaskManager.class);
+
+    @Override
+    public void afterPropertiesSet() throws Exception {
+        logger.info("Kylin service is starting.....");
+
+        runInitialTasks();
+    }
+
+    private void runInitialTasks() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        String initTasks = kylinConfig.getInitTasks();
+        if (!StringUtils.isEmpty(initTasks)) {
+            String[] taskClasses = initTasks.split(",");
+            for (String taskClass : taskClasses) {
+                try {
+                    InitialTask task = (InitialTask) Class.forName(taskClass).newInstance();
+                    logger.info("Running task: " + taskClass);
+                    task.execute();
+                } catch (Exception e) {
+                    logger.error("Initial task failed: " + taskClass, e);
+                }
+            }
+            logger.info("All initial tasks finished.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/model/ColumnMeta.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/model/ColumnMeta.java b/server-base/src/main/java/org/apache/kylin/rest/model/ColumnMeta.java
new file mode 100644
index 0000000..15afb2e
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/model/ColumnMeta.java
@@ -0,0 +1,264 @@
+/*
+ * 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.rest.model;
+
+import java.io.Serializable;
+
+/**
+ */
+public class ColumnMeta implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private String TABLE_CAT;
+    private String TABLE_SCHEM;
+    private String TABLE_NAME;
+    private String COLUMN_NAME;
+    private int DATA_TYPE;
+    private String TYPE_NAME;
+    private int COLUMN_SIZE;
+    private int BUFFER_LENGTH;
+    private int DECIMAL_DIGITS;
+    private int NUM_PREC_RADIX;
+    private int NULLABLE;
+    private String REMARKS;
+    private String COLUMN_DEF;
+    private int SQL_DATA_TYPE;
+    private int SQL_DATETIME_SUB;
+    private int CHAR_OCTET_LENGTH;
+    private int ORDINAL_POSITION;
+    private String IS_NULLABLE;
+    private String SCOPE_CATLOG;
+    private String SCOPE_SCHEMA;
+    private String SCOPE_TABLE;
+    private short SOURCE_DATA_TYPE;
+    private String IS_AUTOINCREMENT;
+
+    public ColumnMeta() {
+    }
+
+    public ColumnMeta(String tABLE_CAT, String tABLE_SCHEM, String tABLE_NAME, String cOLUMN_NAME, int dATA_TYPE, String tYPE_NAME, int cOLUMN_SIZE, int bUFFER_LENGTH, int dECIMAL_DIGITS, int nUM_PREC_RADIX, int nULLABLE, String rEMARKS, String cOLUMN_DEF, int sQL_DATA_TYPE, int sQL_DATETIME_SUB, int cHAR_OCTET_LENGTH, int oRDINAL_POSITION, String iS_NULLABLE, String sCOPE_CATLOG, String sCOPE_SCHEMA, String sCOPE_TABLE, short sOURCE_DATA_TYPE, String iS_AUTOINCREMENT) {
+        super();
+        TABLE_CAT = tABLE_CAT;
+        TABLE_SCHEM = tABLE_SCHEM;
+        TABLE_NAME = tABLE_NAME;
+        COLUMN_NAME = cOLUMN_NAME;
+        DATA_TYPE = dATA_TYPE;
+        TYPE_NAME = tYPE_NAME;
+        COLUMN_SIZE = cOLUMN_SIZE;
+        BUFFER_LENGTH = bUFFER_LENGTH;
+        DECIMAL_DIGITS = dECIMAL_DIGITS;
+        NUM_PREC_RADIX = nUM_PREC_RADIX;
+        NULLABLE = nULLABLE;
+        REMARKS = rEMARKS;
+        COLUMN_DEF = cOLUMN_DEF;
+        SQL_DATA_TYPE = sQL_DATA_TYPE;
+        SQL_DATETIME_SUB = sQL_DATETIME_SUB;
+        CHAR_OCTET_LENGTH = cHAR_OCTET_LENGTH;
+        ORDINAL_POSITION = oRDINAL_POSITION;
+        IS_NULLABLE = iS_NULLABLE;
+        SCOPE_CATLOG = sCOPE_CATLOG;
+        SCOPE_SCHEMA = sCOPE_SCHEMA;
+        SCOPE_TABLE = sCOPE_TABLE;
+        SOURCE_DATA_TYPE = sOURCE_DATA_TYPE;
+        IS_AUTOINCREMENT = iS_AUTOINCREMENT;
+    }
+
+    public String getTABLE_CAT() {
+        return TABLE_CAT;
+    }
+
+    public void setTABLE_CAT(String tABLE_CAT) {
+        TABLE_CAT = tABLE_CAT;
+    }
+
+    public String getTABLE_SCHEM() {
+        return TABLE_SCHEM;
+    }
+
+    public void setTABLE_SCHEM(String tABLE_SCHEM) {
+        TABLE_SCHEM = tABLE_SCHEM;
+    }
+
+    public String getTABLE_NAME() {
+        return TABLE_NAME;
+    }
+
+    public void setTABLE_NAME(String tABLE_NAME) {
+        TABLE_NAME = tABLE_NAME;
+    }
+
+    public String getCOLUMN_NAME() {
+        return COLUMN_NAME;
+    }
+
+    public void setCOLUMN_NAME(String cOLUMN_NAME) {
+        COLUMN_NAME = cOLUMN_NAME;
+    }
+
+    public int getDATA_TYPE() {
+        return DATA_TYPE;
+    }
+
+    public void setDATA_TYPE(int dATA_TYPE) {
+        DATA_TYPE = dATA_TYPE;
+    }
+
+    public String getTYPE_NAME() {
+        return TYPE_NAME;
+    }
+
+    public void setTYPE_NAME(String tYPE_NAME) {
+        TYPE_NAME = tYPE_NAME;
+    }
+
+    public int getCOLUMN_SIZE() {
+        return COLUMN_SIZE;
+    }
+
+    public void setCOLUMN_SIZE(int cOLUMN_SIZE) {
+        COLUMN_SIZE = cOLUMN_SIZE;
+    }
+
+    public int getBUFFER_LENGTH() {
+        return BUFFER_LENGTH;
+    }
+
+    public void setBUFFER_LENGTH(int bUFFER_LENGTH) {
+        BUFFER_LENGTH = bUFFER_LENGTH;
+    }
+
+    public int getDECIMAL_DIGITS() {
+        return DECIMAL_DIGITS;
+    }
+
+    public void setDECIMAL_DIGITS(int dECIMAL_DIGITS) {
+        DECIMAL_DIGITS = dECIMAL_DIGITS;
+    }
+
+    public int getNUM_PREC_RADIX() {
+        return NUM_PREC_RADIX;
+    }
+
+    public void setNUM_PREC_RADIX(int nUM_PREC_RADIX) {
+        NUM_PREC_RADIX = nUM_PREC_RADIX;
+    }
+
+    public int getNULLABLE() {
+        return NULLABLE;
+    }
+
+    public void setNULLABLE(int nULLABLE) {
+        NULLABLE = nULLABLE;
+    }
+
+    public String getREMARKS() {
+        return REMARKS;
+    }
+
+    public void setREMARKS(String rEMARKS) {
+        REMARKS = rEMARKS;
+    }
+
+    public String getCOLUMN_DEF() {
+        return COLUMN_DEF;
+    }
+
+    public void setCOLUMN_DEF(String cOLUMN_DEF) {
+        COLUMN_DEF = cOLUMN_DEF;
+    }
+
+    public int getSQL_DATA_TYPE() {
+        return SQL_DATA_TYPE;
+    }
+
+    public void setSQL_DATA_TYPE(int sQL_DATA_TYPE) {
+        SQL_DATA_TYPE = sQL_DATA_TYPE;
+    }
+
+    public int getSQL_DATETIME_SUB() {
+        return SQL_DATETIME_SUB;
+    }
+
+    public void setSQL_DATETIME_SUB(int sQL_DATETIME_SUB) {
+        SQL_DATETIME_SUB = sQL_DATETIME_SUB;
+    }
+
+    public int getCHAR_OCTET_LENGTH() {
+        return CHAR_OCTET_LENGTH;
+    }
+
+    public void setCHAR_OCTET_LENGTH(int cHAR_OCTET_LENGTH) {
+        CHAR_OCTET_LENGTH = cHAR_OCTET_LENGTH;
+    }
+
+    public int getORDINAL_POSITION() {
+        return ORDINAL_POSITION;
+    }
+
+    public void setORDINAL_POSITION(int oRDINAL_POSITION) {
+        ORDINAL_POSITION = oRDINAL_POSITION;
+    }
+
+    public String getIS_NULLABLE() {
+        return IS_NULLABLE;
+    }
+
+    public void setIS_NULLABLE(String iS_NULLABLE) {
+        IS_NULLABLE = iS_NULLABLE;
+    }
+
+    public String getSCOPE_CATLOG() {
+        return SCOPE_CATLOG;
+    }
+
+    public void setSCOPE_CATLOG(String sCOPE_CATLOG) {
+        SCOPE_CATLOG = sCOPE_CATLOG;
+    }
+
+    public String getSCOPE_SCHEMA() {
+        return SCOPE_SCHEMA;
+    }
+
+    public void setSCOPE_SCHEMA(String sCOPE_SCHEMA) {
+        SCOPE_SCHEMA = sCOPE_SCHEMA;
+    }
+
+    public String getSCOPE_TABLE() {
+        return SCOPE_TABLE;
+    }
+
+    public void setSCOPE_TABLE(String sCOPE_TABLE) {
+        SCOPE_TABLE = sCOPE_TABLE;
+    }
+
+    public short getSOURCE_DATA_TYPE() {
+        return SOURCE_DATA_TYPE;
+    }
+
+    public void setSOURCE_DATA_TYPE(short sOURCE_DATA_TYPE) {
+        SOURCE_DATA_TYPE = sOURCE_DATA_TYPE;
+    }
+
+    public String getIS_AUTOINCREMENT() {
+        return IS_AUTOINCREMENT;
+    }
+
+    public void setIS_AUTOINCREMENT(String iS_AUTOINCREMENT) {
+        this.IS_AUTOINCREMENT = iS_AUTOINCREMENT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/model/Query.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/model/Query.java b/server-base/src/main/java/org/apache/kylin/rest/model/Query.java
new file mode 100644
index 0000000..6b2ef5e
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/model/Query.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.rest.model;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class Query {
+
+    private String name;
+    private String project;
+    private String sql;
+    private String description;
+
+    public Query() {
+    }
+
+    public Query(String name, String project, String sql, String description) {
+        super();
+        this.name = name;
+        this.project = project;
+        this.sql = sql;
+        this.description = description;
+    }
+
+    public String getId() {
+        return String.valueOf(this.hashCode());
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+    public String getSql() {
+        return sql;
+    }
+
+    public void setSql(String sql) {
+        this.sql = sql;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((description == null) ? 0 : description.hashCode());
+        result = prime * result + ((name == null) ? 0 : name.hashCode());
+        result = prime * result + ((project == null) ? 0 : project.hashCode());
+        result = prime * result + ((sql == null) ? 0 : sql.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Query other = (Query) obj;
+        if (description == null) {
+            if (other.description != null)
+                return false;
+        } else if (!description.equals(other.description))
+            return false;
+        if (name == null) {
+            if (other.name != null)
+                return false;
+        } else if (!name.equals(other.name))
+            return false;
+        if (project == null) {
+            if (other.project != null)
+                return false;
+        } else if (!project.equals(other.project))
+            return false;
+        if (sql == null) {
+            if (other.sql != null)
+                return false;
+        } else if (!sql.equals(other.sql))
+            return false;
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/model/SelectedColumnMeta.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/model/SelectedColumnMeta.java b/server-base/src/main/java/org/apache/kylin/rest/model/SelectedColumnMeta.java
new file mode 100644
index 0000000..95582e4
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/model/SelectedColumnMeta.java
@@ -0,0 +1,144 @@
+/*
+ * 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.rest.model;
+
+import java.io.Serializable;
+
+/**
+ */
+public class SelectedColumnMeta implements Serializable {
+    public SelectedColumnMeta(boolean isAutoIncrement, boolean isCaseSensitive, boolean isSearchable, boolean isCurrency, int isNullalbe, boolean isSigned, int displaySize, String label, String name, String schemaName, String catelogName, String tableName, int precision, int scale, int columnType, String columnTypeName, boolean isReadOnly, boolean isWritable, boolean isDefinitelyWritable) {
+        super();
+        this.isAutoIncrement = isAutoIncrement;
+        this.isCaseSensitive = isCaseSensitive;
+        this.isSearchable = isSearchable;
+        this.isCurrency = isCurrency;
+        this.isNullable = isNullalbe;
+        this.isSigned = isSigned;
+        this.displaySize = displaySize;
+        this.label = label;
+        this.name = name;
+        this.schemaName = schemaName;
+        this.catelogName = catelogName;
+        this.tableName = tableName;
+        this.precision = precision;
+        this.scale = scale;
+        this.columnType = columnType;
+        this.columnTypeName = columnTypeName;
+        this.isReadOnly = isReadOnly;
+        this.isWritable = isWritable;
+        this.isDefinitelyWritable = isDefinitelyWritable;
+    }
+
+    public boolean isAutoIncrement() {
+        return isAutoIncrement;
+    }
+
+    public boolean isCaseSensitive() {
+        return isCaseSensitive;
+    }
+
+    public boolean isSearchable() {
+        return isSearchable;
+    }
+
+    public boolean isCurrency() {
+        return isCurrency;
+    }
+
+    public int getIsNullable() {
+        return isNullable;
+    }
+
+    public boolean isSigned() {
+        return isSigned;
+    }
+
+    public int getDisplaySize() {
+        return displaySize;
+    }
+
+    public String getLabel() {
+        return label;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public String getCatelogName() {
+        return catelogName;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public int getScale() {
+        return scale;
+    }
+
+    public int getColumnType() {
+        return columnType;
+    }
+
+    public String getColumnTypeName() {
+        return columnTypeName;
+    }
+
+    public boolean isReadOnly() {
+        return isReadOnly;
+    }
+
+    public boolean isWritable() {
+        return isWritable;
+    }
+
+    public boolean isDefinitelyWritable() {
+        return isDefinitelyWritable;
+    }
+
+    private final boolean isAutoIncrement;
+    private final boolean isCaseSensitive;
+    private final boolean isSearchable;
+    private final boolean isCurrency;
+    private final int isNullable;// 0:nonull, 1:nullable, 2: nullableunknown
+    private final boolean isSigned;
+    private final int displaySize;
+    private final String label;// AS keyword
+    private final String name;
+    private final String schemaName;
+    private final String catelogName;
+    private final String tableName;
+    private final int precision;
+    private final int scale;
+    private final int columnType;// as defined in java.sql.Types
+    private final String columnTypeName;
+    private final boolean isReadOnly;
+    private final boolean isWritable;
+    private final boolean isDefinitelyWritable;
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/model/TableMeta.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/model/TableMeta.java b/server-base/src/main/java/org/apache/kylin/rest/model/TableMeta.java
new file mode 100644
index 0000000..9a48851
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/model/TableMeta.java
@@ -0,0 +1,150 @@
+/*
+ * 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.rest.model;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ */
+public class TableMeta implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private String TABLE_CAT;
+    private String TABLE_SCHEM;
+    private String TABLE_NAME;
+    private String TABLE_TYPE;
+    private String REMARKS;
+    private String TYPE_CAT;
+    private String TYPE_SCHEM;
+    private String TYPE_NAME;
+    private String SELF_REFERENCING_COL_NAME;
+    private String REF_GENERATION;
+    private List<ColumnMeta> columns = new ArrayList<ColumnMeta>();
+
+    public TableMeta() {
+    }
+
+    public TableMeta(String tABLE_CAT, String tABLE_SCHEM, String tABLE_NAME, String tABLE_TYPE, String rEMARKS, String tYPE_CAT, String tYPE_SCHEM, String tYPE_NAME, String sELF_REFERENCING_COL_NAME, String rEF_GENERATION) {
+        super();
+        TABLE_CAT = tABLE_CAT;
+        TABLE_SCHEM = tABLE_SCHEM;
+        TABLE_NAME = tABLE_NAME;
+        TABLE_TYPE = tABLE_TYPE;
+        REMARKS = rEMARKS;
+        TYPE_CAT = tYPE_CAT;
+        TYPE_SCHEM = tYPE_SCHEM;
+        TYPE_NAME = tYPE_NAME;
+        SELF_REFERENCING_COL_NAME = sELF_REFERENCING_COL_NAME;
+        REF_GENERATION = rEF_GENERATION;
+    }
+
+    public String getTABLE_CAT() {
+        return TABLE_CAT;
+    }
+
+    public void setTABLE_CAT(String tABLE_CAT) {
+        TABLE_CAT = tABLE_CAT;
+    }
+
+    public String getTABLE_SCHEM() {
+        return TABLE_SCHEM;
+    }
+
+    public void setTABLE_SCHEM(String tABLE_SCHEM) {
+        TABLE_SCHEM = tABLE_SCHEM;
+    }
+
+    public String getTABLE_NAME() {
+        return TABLE_NAME;
+    }
+
+    public void setTABLE_NAME(String tABLE_NAME) {
+        TABLE_NAME = tABLE_NAME;
+    }
+
+    public String getTABLE_TYPE() {
+        return TABLE_TYPE;
+    }
+
+    public void setTABLE_TYPE(String tABLE_TYPE) {
+        TABLE_TYPE = tABLE_TYPE;
+    }
+
+    public String getREMARKS() {
+        return REMARKS;
+    }
+
+    public void setREMARKS(String rEMARKS) {
+        REMARKS = rEMARKS;
+    }
+
+    public String getTYPE_CAT() {
+        return TYPE_CAT;
+    }
+
+    public void setTYPE_CAT(String tYPE_CAT) {
+        TYPE_CAT = tYPE_CAT;
+    }
+
+    public String getTYPE_SCHEM() {
+        return TYPE_SCHEM;
+    }
+
+    public void setTYPE_SCHEM(String tYPE_SCHEM) {
+        TYPE_SCHEM = tYPE_SCHEM;
+    }
+
+    public String getTYPE_NAME() {
+        return TYPE_NAME;
+    }
+
+    public void setTYPE_NAME(String tYPE_NAME) {
+        TYPE_NAME = tYPE_NAME;
+    }
+
+    public String getSELF_REFERENCING_COL_NAME() {
+        return SELF_REFERENCING_COL_NAME;
+    }
+
+    public void setSELF_REFERENCING_COL_NAME(String sELF_REFERENCING_COL_NAME) {
+        SELF_REFERENCING_COL_NAME = sELF_REFERENCING_COL_NAME;
+    }
+
+    public String getREF_GENERATION() {
+        return REF_GENERATION;
+    }
+
+    public void setREF_GENERATION(String rEF_GENERATION) {
+        REF_GENERATION = rEF_GENERATION;
+    }
+
+    public List<ColumnMeta> getColumns() {
+        return columns;
+    }
+
+    public void setColumns(List<ColumnMeta> columns) {
+        this.columns = columns;
+    }
+
+    public void addColumn(ColumnMeta column) {
+        this.columns.add(column);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/AccessRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/AccessRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/AccessRequest.java
new file mode 100644
index 0000000..0ff57b2
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/AccessRequest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author xduo
+ * 
+ */
+public class AccessRequest {
+
+    private Long accessEntryId;
+    private String permission;
+    private String sid;
+    private boolean principal;
+
+    public AccessRequest() {
+    }
+
+    public Long getAccessEntryId() {
+        return accessEntryId;
+    }
+
+    public void setAccessEntryId(Long accessEntryId) {
+        this.accessEntryId = accessEntryId;
+    }
+
+    public String getPermission() {
+        return permission;
+    }
+
+    public void setPermission(String permission) {
+        this.permission = permission;
+    }
+
+    public String getSid() {
+        return sid;
+    }
+
+    public void setSid(String sid) {
+        this.sid = sid;
+    }
+
+    public boolean isPrincipal() {
+        return principal;
+    }
+
+    public void setPrincipal(boolean principal) {
+        this.principal = principal;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/CardinalityRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/CardinalityRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/CardinalityRequest.java
new file mode 100644
index 0000000..aaf0537
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/CardinalityRequest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.rest.request;
+
+/**
+ * @author jianliu
+ * 
+ */
+public class CardinalityRequest {
+    private int delimiter;
+
+    private int format;
+
+    /**
+     * @return the delimiter
+     */
+    public String getDelimiter() {
+        switch (delimiter) {
+        case 0:
+            return null;
+        case 1:
+            return "177";
+        case 2:
+            return "t";
+        default:
+            return null;
+        }
+    }
+
+    /**
+     * @param delimiter
+     *            the delimiter to set
+     */
+    public void setDelimiter(int delimiter) {
+        this.delimiter = delimiter;
+    }
+
+    /**
+     * @return the format
+     */
+    public String getFormat() {
+        switch (format) {
+        case 0:
+            return null;
+        case 1:
+            return "text";
+        case 2:
+            return "sequence";
+        default:
+            return null;
+        }
+    }
+
+    /**
+     * @param format
+     *            the format to set
+     */
+    public void setFormat(int format) {
+        this.format = format;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/CreateProjectRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/CreateProjectRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/CreateProjectRequest.java
new file mode 100644
index 0000000..71cd1c4
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/CreateProjectRequest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.rest.request;
+
+/**
+ */
+public class CreateProjectRequest {
+    private String name;
+    private String description;
+
+    public CreateProjectRequest() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/CubeRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
new file mode 100644
index 0000000..fb9952b
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.rest.request;
+
+public class CubeRequest {
+
+    private String uuid;
+    private String cubeName;
+    private String cubeDescData;
+    private String streamingData;
+    private String kafkaData;
+    private boolean successful;
+    private String message;
+    private String project;
+    private String streamingCube;
+
+    public String getUuid() {
+        return uuid;
+    }
+
+    public void setUuid(String uuid) {
+        this.uuid = uuid;
+    }
+
+    /**
+     * @return the message
+     */
+    public String getMessage() {
+        return message;
+    }
+
+    /**
+     * @param message
+     *            the message to set
+     */
+    public void setMessage(String message) {
+        this.message = message;
+    }
+
+    /**
+     * @return the status
+     */
+    public boolean getSuccessful() {
+        return successful;
+    }
+
+    /**
+     * @param status
+     *            the status to set
+     */
+    public void setSuccessful(boolean status) {
+        this.successful = status;
+    }
+
+    public CubeRequest() {
+    }
+
+    public CubeRequest(String cubeName, String cubeDescData) {
+        this.cubeName = cubeName;
+        this.cubeDescData = cubeDescData;
+    }
+
+    public String getCubeDescData() {
+        return cubeDescData;
+    }
+
+    public void setCubeDescData(String cubeDescData) {
+        this.cubeDescData = cubeDescData;
+    }
+
+    /**
+     * @return the cubeDescName
+     */
+    public String getCubeName() {
+        return cubeName;
+    }
+
+    /**
+     * @param cubeName
+     *            the cubeDescName to set
+     */
+    public void setCubeName(String cubeName) {
+        this.cubeName = cubeName;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+    public String getStreamingCube() {
+        return streamingCube;
+    }
+
+    public void setStreamingCube(String streamingCube) {
+        this.streamingCube = streamingCube;
+    }
+
+    public String getStreamingData() {
+        return streamingData;
+    }
+
+    public void setStreamingData(String streamingData) {
+        this.streamingData = streamingData;
+    }
+
+    public String getKafkaData() {
+        return kafkaData;
+    }
+
+    public void setKafkaData(String kafkaData) {
+        this.kafkaData = kafkaData;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/CubeSegmentRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/CubeSegmentRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/CubeSegmentRequest.java
new file mode 100644
index 0000000..17122ad
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/CubeSegmentRequest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.rest.request;
+
+public class CubeSegmentRequest {
+
+    private String uuid;
+    private String cubeName;
+    private String cubeSegmentData;
+    private boolean successful;
+    private String message;
+    private String project;
+
+    public String getUuid() {
+        return uuid;
+    }
+
+    public void setUuid(String uuid) {
+        this.uuid = uuid;
+    }
+
+    /**
+     * @return the message
+     */
+    public String getMessage() {
+        return message;
+    }
+
+    /**
+     * @param message
+     *            the message to set
+     */
+    public void setMessage(String message) {
+        this.message = message;
+    }
+
+    /**
+     * @return the status
+     */
+    public boolean getSuccessful() {
+        return successful;
+    }
+
+    /**
+     * @param status
+     *            the status to set
+     */
+    public void setSuccessful(boolean status) {
+        this.successful = status;
+    }
+
+    public CubeSegmentRequest() {
+    }
+
+    public CubeSegmentRequest(String cubeName, String cubeDescData) {
+        this.cubeName = cubeName;
+        this.cubeSegmentData = cubeDescData;
+    }
+
+    public String getCubeSegmentData() {
+        return cubeSegmentData;
+    }
+
+    public void setCubeSegmentData(String cubeSegmentData) {
+        this.cubeSegmentData = cubeSegmentData;
+    }
+
+    /**
+     * @return the cubeDescName
+     */
+    public String getCubeName() {
+        return cubeName;
+    }
+
+    /**
+     * @param cubeName
+     *            the cubeDescName to set
+     */
+    public void setCubeName(String cubeName) {
+        this.cubeName = cubeName;
+    }
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/ExternalFilterRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/ExternalFilterRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/ExternalFilterRequest.java
new file mode 100644
index 0000000..887e9ef
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/ExternalFilterRequest.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.rest.request;
+
+public class ExternalFilterRequest {
+
+    private String project;
+
+    private String extFilter;
+
+    private boolean successful;
+
+    private String message;
+
+    public String getProject() {
+        return project;
+    }
+
+    public void setProject(String project) {
+        this.project = project;
+    }
+
+    public boolean isSuccessful() {
+        return successful;
+    }
+
+    public String getExtFilter() {
+        return extFilter;
+    }
+
+    public void setExtFilter(String extFilter) {
+        this.extFilter = extFilter;
+    }
+
+    public void setSuccessful(boolean successful) {
+        this.successful = successful;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+    public void setMessage(String message) {
+        this.message = message;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
new file mode 100644
index 0000000..c529360
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/HiveTableRequest.java
@@ -0,0 +1,22 @@
+package org.apache.kylin.rest.request;
+
+/**
+ * Created by kangkaisen on 16/5/21.
+ */
+public class HiveTableRequest {
+
+    public HiveTableRequest() {
+
+    }
+
+    private boolean calculate = true;
+
+    public boolean isCalculate() {
+        return calculate;
+    }
+
+    public void setCalculate(boolean calculate) {
+        this.calculate = calculate;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/1a124e68/server-base/src/main/java/org/apache/kylin/rest/request/JobBuildRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/JobBuildRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/JobBuildRequest.java
new file mode 100644
index 0000000..a8ffe5f
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/JobBuildRequest.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.rest.request;
+
+public class JobBuildRequest {
+
+    private long startTime;
+
+    private long endTime;
+
+    private String buildType;
+
+    private boolean force;
+
+    @Deprecated
+    private boolean forceMergeEmptySegment = false;
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+    }
+
+    public String getBuildType() {
+        return buildType;
+    }
+
+    public void setBuildType(String buildType) {
+        this.buildType = buildType;
+    }
+
+    public boolean isForce() {
+        return force;
+    }
+
+    public void setForce(boolean force) {
+        this.force = force;
+    }
+
+    @Deprecated
+    public boolean isForceMergeEmptySegment() {
+        return forceMergeEmptySegment;
+    }
+
+    @Deprecated
+    public void setForceMergeEmptySegment(boolean forceMergeEmptySegment) {
+        this.forceMergeEmptySegment = forceMergeEmptySegment;
+    }
+}