You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by lu...@apache.org on 2015/01/07 15:46:23 UTC

[02/51] [partial] incubator-kylin git commit: migrate repo from github.com to apache git

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/AclService.java b/server/src/main/java/com/kylinolap/rest/service/AclService.java
new file mode 100644
index 0000000..0873dd2
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/AclService.java
@@ -0,0 +1,465 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.acls.domain.AccessControlEntryImpl;
+import org.springframework.security.acls.domain.AclAuthorizationStrategy;
+import org.springframework.security.acls.domain.AclImpl;
+import org.springframework.security.acls.domain.AuditLogger;
+import org.springframework.security.acls.domain.GrantedAuthoritySid;
+import org.springframework.security.acls.domain.ObjectIdentityImpl;
+import org.springframework.security.acls.domain.PermissionFactory;
+import org.springframework.security.acls.domain.PrincipalSid;
+import org.springframework.security.acls.model.AccessControlEntry;
+import org.springframework.security.acls.model.Acl;
+import org.springframework.security.acls.model.AlreadyExistsException;
+import org.springframework.security.acls.model.ChildrenExistException;
+import org.springframework.security.acls.model.MutableAcl;
+import org.springframework.security.acls.model.MutableAclService;
+import org.springframework.security.acls.model.NotFoundException;
+import org.springframework.security.acls.model.ObjectIdentity;
+import org.springframework.security.acls.model.PermissionGrantingStrategy;
+import org.springframework.security.acls.model.Sid;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.security.util.FieldUtils;
+import org.springframework.stereotype.Component;
+import org.springframework.util.Assert;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.HBaseConnection;
+import com.kylinolap.rest.util.Serializer;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("aclService")
+public class AclService implements MutableAclService {
+
+    private static final Logger logger = LoggerFactory.getLogger(AclService.class);
+
+    public static final String ACL_INFO_FAMILY = "i";
+    public static final String ACL_ACES_FAMILY = "a";
+    private static final String DEFAULT_TABLE_PREFIX = "kylin_metadata";
+    private static final String ACL_TABLE_NAME = "_acl";
+    private static final String ACL_INFO_FAMILY_TYPE_COLUMN = "t";
+    private static final String ACL_INFO_FAMILY_OWNER_COLUMN = "o";
+    private static final String ACL_INFO_FAMILY_PARENT_COLUMN = "p";
+    private static final String ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN = "i";
+
+    private Serializer<SidInfo> sidSerializer = new Serializer<SidInfo>(SidInfo.class);
+    private Serializer<DomainObjectInfo> domainObjSerializer = new Serializer<DomainObjectInfo>(DomainObjectInfo.class);
+    private Serializer<AceInfo> aceSerializer = new Serializer<AceInfo>(AceInfo.class);
+
+    private String hbaseUrl = null;
+    private String tableNameBase = null;
+    private String aclTableName = null;
+
+    private final Field fieldAces = FieldUtils.getField(AclImpl.class, "aces");
+    private final Field fieldAcl = FieldUtils.getField(AccessControlEntryImpl.class, "acl");
+
+    @Autowired
+    protected PermissionGrantingStrategy permissionGrantingStrategy;
+
+    @Autowired
+    protected PermissionFactory aclPermissionFactory;
+
+    @Autowired
+    protected AclAuthorizationStrategy aclAuthorizationStrategy;
+
+    @Autowired
+    protected AuditLogger auditLogger;
+
+    public AclService() throws IOException {
+        String metadataUrl = KylinConfig.getInstanceFromEnv().getMetadataUrl();
+        // split TABLE@HBASE_URL
+        int cut = metadataUrl.indexOf('@');
+        tableNameBase = cut < 0 ? DEFAULT_TABLE_PREFIX : metadataUrl.substring(0, cut);
+        hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
+        aclTableName = tableNameBase + ACL_TABLE_NAME;
+
+        fieldAces.setAccessible(true);
+        fieldAcl.setAccessible(true);
+
+        HBaseConnection.createHTableIfNeeded(hbaseUrl, aclTableName, ACL_INFO_FAMILY, ACL_ACES_FAMILY);
+    }
+
+    @Override
+    public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
+        List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+
+            Scan scan = new Scan();
+            SingleColumnValueFilter parentFilter = new SingleColumnValueFilter(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), CompareOp.EQUAL, domainObjSerializer.serialize(new DomainObjectInfo(parentIdentity)));
+            parentFilter.setFilterIfMissing(true);
+            scan.setFilter(parentFilter);
+
+            ResultScanner scanner = htable.getScanner(scan);
+            for (Result result = scanner.next(); result != null; result = scanner.next()) {
+                String id = Bytes.toString(result.getRow());
+                String type = Bytes.toString(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN)));
+
+                oids.add(new ObjectIdentityImpl(type, id));
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return oids;
+    }
+
+    @Override
+    public Acl readAclById(ObjectIdentity object) throws NotFoundException {
+        Map<ObjectIdentity, Acl> aclsMap = readAclsById(Arrays.asList(object), null);
+        //        Assert.isTrue(aclsMap.containsKey(object), "There should have been an Acl entry for ObjectIdentity " + object);
+
+        return aclsMap.get(object);
+    }
+
+    @Override
+    public Acl readAclById(ObjectIdentity object, List<Sid> sids) throws NotFoundException {
+        Map<ObjectIdentity, Acl> aclsMap = readAclsById(Arrays.asList(object), sids);
+        Assert.isTrue(aclsMap.containsKey(object), "There should have been an Acl entry for ObjectIdentity " + object);
+
+        return aclsMap.get(object);
+    }
+
+    @Override
+    public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> objects) throws NotFoundException {
+        return readAclsById(objects, null);
+    }
+
+    @Override
+    public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
+        Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
+        HTableInterface htable = null;
+        Result result = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+
+            for (ObjectIdentity oid : oids) {
+                result = htable.get(new Get(Bytes.toBytes(String.valueOf(oid.getIdentifier()))));
+
+                if (null != result && !result.isEmpty()) {
+                    SidInfo owner = sidSerializer.deserialize(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN)));
+                    Sid ownerSid = (null == owner) ? null : (owner.isPrincipal() ? new PrincipalSid(owner.getSid()) : new GrantedAuthoritySid(owner.getSid()));
+                    boolean entriesInheriting = Bytes.toBoolean(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN)));
+
+                    Acl parentAcl = null;
+                    DomainObjectInfo parentInfo = domainObjSerializer.deserialize(result.getValue(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN)));
+                    if (null != parentInfo) {
+                        ObjectIdentity parentObj = new ObjectIdentityImpl(parentInfo.getType(), parentInfo.getId());
+                        parentAcl = readAclById(parentObj, null);
+                    }
+
+                    AclImpl acl = new AclImpl(oid, oid.getIdentifier(), aclAuthorizationStrategy, permissionGrantingStrategy, parentAcl, null, entriesInheriting, ownerSid);
+                    genAces(sids, result, acl);
+
+                    aclMaps.put(oid, acl);
+                } else {
+                    throw new NotFoundException("Unable to find ACL information for object identity '" + oid + "'");
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return aclMaps;
+    }
+
+    @Override
+    public MutableAcl createAcl(ObjectIdentity objectIdentity) throws AlreadyExistsException {
+        Acl acl = null;
+
+        try {
+            acl = readAclById(objectIdentity);
+        } catch (NotFoundException e) {
+        }
+        if (null != acl) {
+            throw new AlreadyExistsException("ACL of " + objectIdentity + " exists!");
+        }
+
+        Authentication auth = SecurityContextHolder.getContext().getAuthentication();
+        PrincipalSid sid = new PrincipalSid(auth);
+
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
+            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+
+            htable.put(put);
+            htable.flushCommits();
+
+            logger.debug("ACL of " + objectIdentity + " created successfully.");
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return (MutableAcl) readAclById(objectIdentity);
+    }
+
+    @Override
+    public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            Delete delete = new Delete(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
+
+            List<ObjectIdentity> children = findChildren(objectIdentity);
+            if (!deleteChildren && children.size() > 0) {
+                throw new ChildrenExistException("Children exists for " + objectIdentity);
+            }
+
+            for (ObjectIdentity oid : children) {
+                deleteAcl(oid, deleteChildren);
+            }
+
+            htable.delete(delete);
+            htable.flushCommits();
+
+            logger.debug("ACL of " + objectIdentity + " deleted successfully.");
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+    }
+
+    @Override
+    public MutableAcl updateAcl(MutableAcl acl) throws NotFoundException {
+        try {
+            readAclById(acl.getObjectIdentity());
+        } catch (NotFoundException e) {
+            throw e;
+        }
+
+        HTableInterface htable = null;
+        try {
+            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            Delete delete = new Delete(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
+            delete.deleteFamily(Bytes.toBytes(ACL_ACES_FAMILY));
+            htable.delete(delete);
+
+            Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
+
+            if (null != acl.getParentAcl()) {
+                put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+            }
+
+            for (AccessControlEntry ace : acl.getEntries()) {
+                AceInfo aceInfo = new AceInfo(ace);
+                put.add(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+            }
+
+            if (!put.isEmpty()) {
+                htable.put(put);
+                htable.flushCommits();
+
+                logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        } finally {
+            IOUtils.closeQuietly(htable);
+        }
+
+        return (MutableAcl) readAclById(acl.getObjectIdentity());
+    }
+
+    private void genAces(List<Sid> sids, Result result, AclImpl acl) throws JsonParseException, JsonMappingException, IOException {
+        List<AceInfo> aceInfos = new ArrayList<AceInfo>();
+        if (null != sids) {
+            // Just return aces in sids
+            for (Sid sid : sids) {
+                String sidName = null;
+                if (sid instanceof PrincipalSid) {
+                    sidName = ((PrincipalSid) sid).getPrincipal();
+                } else if (sid instanceof GrantedAuthoritySid) {
+                    sidName = ((GrantedAuthoritySid) sid).getGrantedAuthority();
+                }
+
+                AceInfo aceInfo = aceSerializer.deserialize(result.getValue(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(sidName)));
+                if (null != aceInfo) {
+                    aceInfos.add(aceInfo);
+                }
+            }
+        } else {
+            NavigableMap<byte[], byte[]> familyMap = result.getFamilyMap(Bytes.toBytes(ACL_ACES_FAMILY));
+            for (byte[] qualifier : familyMap.keySet()) {
+                AceInfo aceInfo = aceSerializer.deserialize(familyMap.get(qualifier));
+
+                if (null != aceInfo) {
+                    aceInfos.add(aceInfo);
+                }
+            }
+        }
+
+        List<AccessControlEntry> newAces = new ArrayList<AccessControlEntry>();
+        for (int i = 0; i < aceInfos.size(); i++) {
+            AceInfo aceInfo = aceInfos.get(i);
+
+            if (null != aceInfo) {
+                Sid sid = aceInfo.getSidInfo().isPrincipal() ? new PrincipalSid(aceInfo.getSidInfo().getSid()) : new GrantedAuthoritySid(aceInfo.getSidInfo().getSid());
+                AccessControlEntry ace = new AccessControlEntryImpl(Long.valueOf(i), acl, sid, aclPermissionFactory.buildFromMask(aceInfo.getPermissionMask()), true, false, false);
+                newAces.add(ace);
+            }
+        }
+
+        this.setAces(acl, newAces);
+    }
+
+    private void setAces(AclImpl acl, List<AccessControlEntry> aces) {
+        try {
+            fieldAces.set(acl, aces);
+        } catch (IllegalAccessException e) {
+            throw new IllegalStateException("Could not set AclImpl entries", e);
+        }
+    }
+
+    protected static class DomainObjectInfo {
+        private String id;
+        private String type;
+
+        public DomainObjectInfo() {
+        }
+
+        public DomainObjectInfo(ObjectIdentity oid) {
+            super();
+            this.id = (String) oid.getIdentifier();
+            this.type = oid.getType();
+        }
+
+        public Serializable getId() {
+            return id;
+        }
+
+        public void setId(String id) {
+            this.id = id;
+        }
+
+        public String getType() {
+            return type;
+        }
+
+        public void setType(String type) {
+            this.type = type;
+        }
+    }
+
+    protected static class SidInfo {
+        private String sid;
+        private boolean isPrincipal;
+
+        public SidInfo() {
+        }
+
+        public SidInfo(Sid sid) {
+            if (sid instanceof PrincipalSid) {
+                this.sid = ((PrincipalSid) sid).getPrincipal();
+                this.isPrincipal = true;
+            } else if (sid instanceof GrantedAuthoritySid) {
+                this.sid = ((GrantedAuthoritySid) sid).getGrantedAuthority();
+                this.isPrincipal = false;
+            }
+        }
+
+        public String getSid() {
+            return sid;
+        }
+
+        public void setSid(String sid) {
+            this.sid = sid;
+        }
+
+        public boolean isPrincipal() {
+            return isPrincipal;
+        }
+
+        public void setPrincipal(boolean isPrincipal) {
+            this.isPrincipal = isPrincipal;
+        }
+    }
+
+    protected static class AceInfo {
+        private SidInfo sidInfo;
+        private int permissionMask;
+
+        public AceInfo() {
+        }
+
+        public AceInfo(AccessControlEntry ace) {
+            super();
+            this.sidInfo = new SidInfo(ace.getSid());
+            this.permissionMask = ace.getPermission().getMask();
+        }
+
+        public SidInfo getSidInfo() {
+            return sidInfo;
+        }
+
+        public void setSidInfo(SidInfo sidInfo) {
+            this.sidInfo = sidInfo;
+        }
+
+        public int getPermissionMask() {
+            return permissionMask;
+        }
+
+        public void setPermissionMask(int permissionMask) {
+            this.permissionMask = permissionMask;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/AdminService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/AdminService.java b/server/src/main/java/com/kylinolap/rest/service/AdminService.java
new file mode 100644
index 0000000..f7a0211
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/AdminService.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.job.hadoop.cube.StorageCleanupJob;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+
+/**
+ * @author jianliu
+ */
+@Component("adminService")
+public class AdminService extends BasicService {
+    private static final Logger logger = LoggerFactory.getLogger(AdminService.class);
+
+    /**
+     * Get Java Env info as string
+     *
+     * @return
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public String getEnv() {
+        logger.debug("Get Kylin Runtime environment");
+        PropertiesConfiguration tempConfig = new PropertiesConfiguration();
+
+        // Add Java Env
+
+        try {
+            String content = "";
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            // env
+            Map<String, String> env = System.getenv();
+
+            for (Map.Entry<String, String> entry : env.entrySet()) {
+                tempConfig.addProperty(entry.getKey(), entry.getValue());
+            }
+
+            // properties
+            Properties proterties = System.getProperties();
+
+            for(Map.Entry<Object,Object> entry : proterties.entrySet())
+            {
+                tempConfig.setProperty((String) entry.getKey(), entry.getValue());
+            }
+
+
+            // do save
+            tempConfig.save(baos);
+            content = baos.toString();
+            return content;
+        } catch (ConfigurationException e) {
+            throw new InternalErrorException("Failed to get Kylin env Config", e);
+        }
+    }
+
+    /**
+     * Get Java config info as String
+     *
+     * @return
+     */
+    // @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public String getConfigAsString() {
+        logger.debug("Get Kylin Runtime Config");
+
+        try {
+            return KylinConfig.getInstanceFromEnv().getConfigAsString();
+        } catch (IOException e) {
+            throw new InternalErrorException("Failed to get Kylin Runtime Config", e);
+        }
+    }
+
+    public void cleanupStorage() {
+        StorageCleanupJob job = new StorageCleanupJob();
+        String[] args = new String[] { "-delete", "true" };
+        try {
+            ToolRunner.run(job, args);
+        } catch (Exception e) {
+            throw new InternalErrorException(e.getMessage(), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/BasicService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/BasicService.java b/server/src/main/java/com/kylinolap/rest/service/BasicService.java
new file mode 100644
index 0000000..812c6aa
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/BasicService.java
@@ -0,0 +1,193 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.charset.Charset;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.sql.DataSource;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.cache.annotation.CacheEvict;
+import org.springframework.cache.annotation.Caching;
+import org.springframework.jdbc.datasource.DriverManagerDataSource;
+
+import com.google.common.io.Files;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.cube.project.ProjectManager;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.job.engine.JobEngineConfig;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.metadata.MetadataManager;
+import com.kylinolap.query.enumerator.OLAPQuery;
+import com.kylinolap.query.relnode.OLAPContext;
+import com.kylinolap.query.schema.OLAPSchemaFactory;
+import com.kylinolap.rest.controller.QueryController;
+
+public abstract class BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(BasicService.class);
+
+    private static ConcurrentMap<String, DataSource> olapDataSources = new ConcurrentHashMap<String, DataSource>();
+
+//    @Autowired
+//    protected JdbcTemplate jdbcTemplate;
+
+    public KylinConfig getConfig() {
+        return KylinConfig.getInstanceFromEnv();
+    }
+
+    public void removeOLAPDataSource(String project) {
+        if (StringUtils.isEmpty(project))
+            throw new IllegalArgumentException("removeOLAPDataSource: project name not given");
+
+        project = ProjectInstance.getNormalizedProjectName(project);
+        olapDataSources.remove(project);
+    }
+
+    public static void resetOLAPDataSources() {
+        // brutal, yet simplest way
+        logger.info("resetOLAPDataSources is called.");
+        olapDataSources = new ConcurrentHashMap<String, DataSource>();
+    }
+
+    public DataSource getOLAPDataSource(String project) {
+
+        project = ProjectInstance.getNormalizedProjectName(project);
+
+        DataSource ret = olapDataSources.get(project);
+        if (ret == null) {
+            logger.debug("Creating a new data source");
+            logger.debug("OLAP data source pointing to " + getConfig());
+
+            File modelJson = OLAPSchemaFactory.createTempOLAPJson(project, getConfig());
+
+            try {
+                List<String> text = Files.readLines(modelJson, Charset.defaultCharset());
+                logger.debug("The new temp olap json is :");
+                for (String line : text)
+                    logger.debug(line);
+            } catch (IOException e) {
+                e.printStackTrace(); // logging failure is not critical
+            }
+
+            DriverManagerDataSource ds = new DriverManagerDataSource();
+            Properties props = new Properties();
+            props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, String.valueOf(KylinConfig.getInstanceFromEnv().getScanThreshold()));
+            ds.setConnectionProperties(props);
+            ds.setDriverClassName("net.hydromatic.optiq.jdbc.Driver");
+            ds.setUrl("jdbc:calcite:model=" + modelJson.getAbsolutePath());
+
+            ret = olapDataSources.putIfAbsent(project, ds);
+            if (ret == null) {
+                ret = ds;
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * Reload changed cube into cache
+     * 
+     * @param name
+     * @throws IOException
+     */
+    @Caching(evict = { @CacheEvict(value = QueryController.SUCCESS_QUERY_CACHE, allEntries = true), @CacheEvict(value = QueryController.EXCEPTION_QUERY_CACHE, allEntries = true) })
+    public void cleanDataCache() {
+        CubeManager.removeInstance(getConfig());
+        ProjectManager.removeInstance(getConfig());
+        BasicService.resetOLAPDataSources();
+    }
+
+    /**
+     * Reload the cube desc with name {name} into cache
+     * 
+     * @param name
+     */
+    public void reloadMetadataCache() {
+        MetadataManager.getInstance(getConfig()).reload();
+    }
+
+    public KylinConfig getKylinConfig() {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+
+        if (kylinConfig == null) {
+            throw new IllegalArgumentException("Failed to load kylin config instance");
+        }
+
+        return kylinConfig;
+    }
+
+    public MetadataManager getMetadataManager() {
+        return MetadataManager.getInstance(getConfig());
+    }
+
+    public CubeManager getCubeManager() {
+        return CubeManager.getInstance(getConfig());
+    }
+
+    public ProjectManager getProjectManager() {
+        return ProjectManager.getInstance(getConfig());
+    }
+
+    public JobManager getJobManager() throws JobException, UnknownHostException {
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+
+        JobEngineConfig engineCntx = new JobEngineConfig(config);
+
+        InetAddress ia = InetAddress.getLocalHost();
+        return new JobManager(ia.getCanonicalHostName(), engineCntx);
+    }
+
+    protected static void close(ResultSet resultSet, Statement stat, Connection conn) {
+        OLAPContext.clearParameter();
+
+        if (resultSet != null)
+            try {
+                resultSet.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        if (stat != null)
+            try {
+                stat.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        if (conn != null)
+            try {
+                conn.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/CubeService.java b/server/src/main/java/com/kylinolap/rest/service/CubeService.java
new file mode 100644
index 0000000..0215504
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/CubeService.java
@@ -0,0 +1,598 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.UnknownHostException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.cache.annotation.CacheEvict;
+import org.springframework.cache.annotation.Caching;
+import org.springframework.security.access.prepost.PostFilter;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.kylinolap.common.KylinConfig;
+import com.kylinolap.common.persistence.ResourceStore;
+import com.kylinolap.common.util.HBaseRegionSizeCalculator;
+import com.kylinolap.common.util.HadoopUtil;
+import com.kylinolap.common.util.JsonUtil;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeManager;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.CubeSegmentStatusEnum;
+import com.kylinolap.cube.CubeStatusEnum;
+import com.kylinolap.cube.cuboid.CuboidCLI;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.job.JobDAO;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.JobInstance.JobStep;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.constant.JobStepStatusEnum;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.job.hadoop.cardinality.HiveColumnCardinalityJob;
+import com.kylinolap.metadata.MetadataConstances;
+import com.kylinolap.metadata.model.cube.CubeDesc;
+import com.kylinolap.metadata.model.schema.ColumnDesc;
+import com.kylinolap.metadata.model.schema.TableDesc;
+import com.kylinolap.metadata.tool.HiveSourceTableLoader;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.controller.QueryController;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.MetricsRequest;
+import com.kylinolap.rest.response.HBaseResponse;
+import com.kylinolap.rest.response.MetricsResponse;
+import com.kylinolap.rest.security.AclPermission;
+
+/**
+ * Stateless & lightweight service facade of cube management functions.
+ *
+ * @author yangli9
+ */
+@Component("cubeMgmtService")
+public class CubeService extends BasicService {
+    private static final String DESC_SUFFIX = "_desc";
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeService.class);
+
+    @Autowired
+    private AccessService accessService;
+
+    @PostFilter(Constant.ACCESS_POST_FILTER_READ)
+    public List<CubeInstance> listAllCubes(final String cubeName, final String projectName) {
+        List<CubeInstance> cubeInstances = null;
+        ProjectInstance project = (null != projectName) ? getProjectManager().getProject(projectName) : null;
+
+        if (null == project) {
+            cubeInstances = getCubeManager().listAllCubes();
+        } else {
+            cubeInstances = getProjectManager().listAllCubes(projectName);
+        }
+
+        List<CubeInstance> filterCubes = new ArrayList<CubeInstance>();
+        for (CubeInstance cubeInstance : cubeInstances) {
+            boolean isCubeMatch = (null == cubeName) || cubeInstance.getName().toLowerCase().contains(cubeName.toLowerCase());
+
+            if (isCubeMatch) {
+                filterCubes.add(cubeInstance);
+            }
+        }
+
+        return filterCubes;
+    }
+
+    public List<CubeInstance> getCubes(final String cubeName, final String projectName, final Integer limit, final Integer offset) {
+        int climit = (null == limit) ? 30 : limit;
+        int coffset = (null == offset) ? 0 : offset;
+
+        List<CubeInstance> cubes;
+        cubes = listAllCubes(cubeName, projectName);
+
+        if (cubes.size() <= coffset) {
+            return Collections.emptyList();
+        }
+
+        if ((cubes.size() - coffset) < climit) {
+            return cubes.subList(coffset, cubes.size());
+        }
+
+        return cubes.subList(coffset, coffset + climit);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public CubeInstance updateCubeCost(String cubeName, int cost) throws IOException, CubeIntegrityException {
+        CubeInstance cube = getCubeManager().getCube(cubeName);
+        if (cube == null) {
+            throw new IOException("Cannot find cube " + cubeName);
+        }
+        if (cube.getCost() == cost) {
+            // Do nothing
+            return cube;
+        }
+        cube.setCost(cost);
+
+        String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+        cube.setOwner(owner);
+
+        return getCubeManager().updateCube(cube);
+    }
+
+    public CubeInstance createCubeAndDesc(String cubeName, String projectName, CubeDesc desc) throws IOException {
+        if (getCubeManager().getCube(cubeName) != null) {
+            throw new InternalErrorException("The cube named " + cubeName + " already exists");
+        }
+
+        String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+        CubeDesc createdDesc = null;
+        CubeInstance createdCube = null;
+
+        createdDesc = getMetadataManager().createCubeDesc(desc);
+
+        if (!createdDesc.getError().isEmpty()) {
+            getMetadataManager().removeCubeDesc(createdDesc);
+            throw new InternalErrorException(createdDesc.getError().get(0));
+        }
+
+        try {
+            int cuboidCount = CuboidCLI.simulateCuboidGeneration(createdDesc);
+            logger.info("New cube " + cubeName + " has " + cuboidCount + " cuboids");
+        } catch (Exception e) {
+            getMetadataManager().removeCubeDesc(createdDesc);
+            throw new InternalErrorException("Failed to deal with the request."+e.getLocalizedMessage(), e);
+        }
+
+        createdCube = getCubeManager().createCube(cubeName, projectName, createdDesc, owner);
+        accessService.init(createdCube, AclPermission.ADMINISTRATION);
+
+        ProjectInstance project = getProjectManager().getProject(projectName);
+        accessService.inherit(createdCube, project);
+
+        return createdCube;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public CubeDesc updateCubeAndDesc(CubeInstance cube, CubeDesc desc, String newProjectName) throws Exception {
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("Cube schema shouldn't be changed with running job.");
+            }
+        }
+
+        if (!cube.getDescriptor().calculateSignature().equals(cube.getDescriptor().getSignature())) {
+            this.releaseAllSegments(cube);
+        }
+
+        CubeDesc updatedCubeDesc = getMetadataManager().updateCubeDesc(desc);
+        if (updatedCubeDesc.getError().size() > 0)
+            return updatedCubeDesc;
+
+        int cuboidCount = CuboidCLI.simulateCuboidGeneration(updatedCubeDesc);
+        logger.info("Updated cube " + cube.getName() + " has " + cuboidCount + " cuboids");
+
+        if (!getProjectManager().isCubeInProject(newProjectName, cube)) {
+            String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+            ProjectInstance newProject = getProjectManager().updateCubeToProject(cube.getName(), newProjectName, owner);
+            accessService.inherit(cube, newProject);
+        }
+
+        return updatedCubeDesc;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public void deleteCube(CubeInstance cube) throws IOException, JobException, CubeIntegrityException {
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("The cube " + cube.getName() + " has running job, please discard it and try again.");
+            }
+        }
+
+        this.releaseAllSegments(cube);
+        getCubeManager().dropCube(cube.getName(), true);
+        accessService.clean(cube, true);
+    }
+
+    public boolean isCubeEditable(CubeInstance ci) {
+        return ci.getStatus() == CubeStatusEnum.DISABLED;
+    }
+
+    public boolean isCubeDescEditable(CubeDesc cd) {
+        List<CubeInstance> list = getCubeManager().getCubesByDesc(cd.getName());
+        if (list.isEmpty()) {
+            return true;
+        }
+        Iterator<CubeInstance> it = list.iterator();
+        while (it.hasNext()) {
+            if (!isCubeEditable(it.next())) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static String getCubeDescNameFromCube(String cubeName) {
+        return cubeName + DESC_SUFFIX;
+    }
+
+    public static String getCubeNameFromDesc(String descName) {
+        if (descName.toLowerCase().endsWith(DESC_SUFFIX)) {
+            return descName.substring(0, descName.toLowerCase().indexOf(DESC_SUFFIX));
+        } else {
+            return descName;
+        }
+    }
+
+    public void reloadCubeCache(String cubeName) {
+        CubeInstance cube = CubeManager.getInstance(this.getConfig()).getCube(cubeName);
+        CubeManager.getInstance(this.getConfig()).loadCubeCache(cube);
+    }
+
+    public void removeCubeCache(String cubeName) {
+        CubeInstance cube = CubeManager.getInstance(this.getConfig()).getCube(cubeName);
+        CubeManager.getInstance(this.getConfig()).removeCubeCache(cube);
+    }
+
+    /**
+     * Stop all jobs belonging to this cube and clean out all segments
+     *
+     * @param cube
+     * @return
+     * @throws IOException
+     * @throws CubeIntegrityException
+     * @throws JobException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
+    @Caching(evict = { @CacheEvict(value = QueryController.SUCCESS_QUERY_CACHE, allEntries = true), @CacheEvict(value = QueryController.EXCEPTION_QUERY_CACHE, allEntries = true) })
+    public CubeInstance purgeCube(CubeInstance cube) throws IOException, CubeIntegrityException, JobException {
+        String cubeName = cube.getName();
+
+        CubeStatusEnum ostatus = cube.getStatus();
+        if (null != ostatus && !CubeStatusEnum.DISABLED.equals(ostatus)) {
+            throw new InternalErrorException("Only disabled cube can be purged, status of " + cubeName + " is " + ostatus);
+        }
+
+        try {
+            this.releaseAllSegments(cube);
+            return cube;
+        } catch (IOException e) {
+            throw e;
+        }
+
+    }
+
+    /**
+     * Update a cube status from ready to disabled.
+     *
+     * @return
+     * @throws CubeIntegrityException
+     * @throws IOException
+     * @throws JobException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
+    @Caching(evict = { @CacheEvict(value = QueryController.SUCCESS_QUERY_CACHE, allEntries = true), @CacheEvict(value = QueryController.EXCEPTION_QUERY_CACHE, allEntries = true) })
+    public CubeInstance disableCube(CubeInstance cube) throws IOException, CubeIntegrityException, JobException {
+        String cubeName = cube.getName();
+
+        CubeStatusEnum ostatus = cube.getStatus();
+        if (null != ostatus && !CubeStatusEnum.READY.equals(ostatus)) {
+            throw new InternalErrorException("Only ready cube can be disabled, status of " + cubeName + " is " + ostatus);
+        }
+
+        cube.setStatus(CubeStatusEnum.DISABLED);
+
+        try {
+            return getCubeManager().updateCube(cube);
+        } catch (IOException e) {
+            cube.setStatus(ostatus);
+            throw e;
+        }
+    }
+
+    /**
+     * Update a cube status from disable to ready.
+     *
+     * @return
+     * @throws CubeIntegrityException
+     * @throws IOException
+     * @throws JobException
+     */
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION')  or hasPermission(#cube, 'MANAGEMENT')")
+    public CubeInstance enableCube(CubeInstance cube) throws IOException, CubeIntegrityException, JobException {
+        String cubeName = cube.getName();
+
+        CubeStatusEnum ostatus = cube.getStatus();
+        if (!cube.getStatus().equals(CubeStatusEnum.DISABLED)) {
+            throw new InternalErrorException("Only disabled cube can be enabled, status of " + cubeName + " is " + ostatus);
+        }
+
+        if (cube.getSegments(CubeSegmentStatusEnum.READY).size() == 0) {
+            throw new InternalErrorException("Cube " + cubeName + " dosen't contain any READY segment");
+        }
+
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("Enable is not allowed with a running job.");
+            }
+        }
+        if (!cube.getDescriptor().calculateSignature().equals(cube.getDescriptor().getSignature())) {
+            this.releaseAllSegments(cube);
+        }
+
+        cube.setStatus(CubeStatusEnum.READY);
+        try {
+            return getCubeManager().updateCube(cube);
+        } catch (IOException e) {
+            cube.setStatus(ostatus);
+            throw e;
+        }
+    }
+
+    public MetricsResponse calculateMetrics(MetricsRequest request) {
+        DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z");
+        List<CubeInstance> cubes = this.getCubeManager().listAllCubes();
+        MetricsResponse metrics = new MetricsResponse();
+        Date startTime = (null == request.getStartTime()) ? new Date(-1) : request.getStartTime();
+        Date endTime = (null == request.getEndTime()) ? new Date() : request.getEndTime();
+        metrics.increase("totalCubes", (float) 0);
+        metrics.increase("totalStorage", (float) 0);
+
+        for (CubeInstance cube : cubes) {
+            Date createdDate = new Date(-1);
+            try {
+                createdDate = (null == cube.getCreateTime()) ? createdDate : format.parse(cube.getCreateTime());
+            } catch (ParseException e) {
+                logger.error("", e);
+            }
+
+            if (createdDate.getTime() > startTime.getTime() && createdDate.getTime() < endTime.getTime()) {
+                metrics.increase("totalCubes");
+            }
+        }
+
+        metrics.increase("aveStorage", (metrics.get("totalCubes") == 0) ? 0 : metrics.get("totalStorage") / metrics.get("totalCubes"));
+
+        return metrics;
+    }
+
+    /**
+     * Calculate size of each region for given table and other info of the
+     * table.
+     *
+     * @param tableName The table name.
+     * @return The HBaseResponse object contains table size, region count. null
+     * if error happens.
+     * @throws IOException Exception when HTable resource is not closed correctly.
+     */
+    public HBaseResponse getHTableInfo(String tableName) throws IOException {
+        // Get HBase storage conf.
+        String hbaseUrl = KylinConfig.getInstanceFromEnv().getStorageUrl();
+        Configuration hconf = HadoopUtil.newHBaseConfiguration(hbaseUrl);
+
+        HTable table = null;
+        HBaseResponse hr = null;
+        long tableSize = 0;
+        int regionCount = 0;
+
+        try {
+            table = new HTable(hconf, tableName);
+
+            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
+            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+
+            for (long s : sizeMap.values()) {
+                tableSize += s;
+            }
+
+            regionCount = sizeMap.size();
+
+            // Set response.
+            hr = new HBaseResponse();
+            hr.setTableSize(tableSize);
+            hr.setRegionCount(regionCount);
+        } finally {
+            if (null != table) {
+                table.close();
+            }
+        }
+
+        return hr;
+    }
+
+    /**
+     * Generate cardinality for table This will trigger a hadoop job and nothing
+     * The result will be merged into table exd info
+     *
+     * @param tableName
+     * @param delimiter
+     * @param format
+     */
+    public void generateCardinality(String tableName, String format, String delimiter) {
+        TableDesc table = getMetadataManager().getTableDesc(tableName);
+        Map<String, String> tableExd = getMetadataManager().getTableDescExd(tableName);
+        if (tableExd == null || table == null) {
+            IllegalArgumentException e = new IllegalArgumentException("Cannot find table descirptor " + tableName);
+            logger.error("Cannot find table descirptor " + tableName, e);
+            throw e;
+        }
+        Map<String, String> exd = getMetadataManager().getTableDescExd(tableName);
+        if (exd == null || !Boolean.valueOf(exd.get(MetadataConstances.TABLE_EXD_STATUS_KEY))) {
+            throw new IllegalArgumentException("Table " + tableName + " does not exist.");
+        }
+        String location = exd.get(MetadataConstances.TABLE_EXD_LOCATION);
+        if (location == null || MetadataConstances.TABLE_EXD_DEFAULT_VALUE.equals(location)) {
+            throw new IllegalArgumentException("Cannot get table " + tableName + " location, the location is " + location);
+        }
+        String inputFormat = exd.get(MetadataConstances.TABLE_EXD_IF);
+        if (inputFormat == null || MetadataConstances.TABLE_EXD_DEFAULT_VALUE.equals(inputFormat)) {
+            throw new IllegalArgumentException("Cannot get table " + tableName + " input format, the format is " + inputFormat);
+        }
+        String delim = exd.get(MetadataConstances.TABLE_EXD_DELIM);
+        if (delimiter != null) {
+            delim = delimiter;
+        }
+        String jarPath = getKylinConfig().getKylinJobJarPath();
+        String outPath = HiveColumnCardinalityJob.OUTPUT_PATH + "/" + tableName;
+        String[] args = null;
+        if (delim == null) {
+            args = new String[] { "-input", location, "-output", outPath, "-iformat", inputFormat };
+        } else {
+            args = new String[] { "-input", location, "-output", outPath, "-iformat", inputFormat, "-idelim", delim };
+        }
+        HiveColumnCardinalityJob job = new HiveColumnCardinalityJob(jarPath, null);
+        int hresult = 0;
+        try {
+            hresult = ToolRunner.run(job, args);
+        } catch (Exception e) {
+            logger.error("Cardinality calculation failed. ", e);
+            throw new IllegalArgumentException("Hadoop job failed with exception ", e);
+        }
+
+        // Get calculate result;
+        if (hresult != 0) {
+            throw new IllegalArgumentException("Hadoop job failed with result " + hresult);
+        }
+        List<String> columns = null;
+        try {
+            columns = job.readLines(new Path(outPath), job.getConf());
+        } catch (IllegalArgumentException e) {
+            logger.error("Failed to resolve cardinality for " + tableName + " from " + outPath, e);
+            return;
+        } catch (Exception e) {
+            logger.error("Failed to resolve cardinality for " + tableName + " from " + outPath, e);
+            return;
+        }
+        StringBuffer cardi = new StringBuffer();
+        ColumnDesc[] cols = table.getColumns();
+        if (columns.isEmpty() || cols.length != columns.size()) {
+            logger.error("The hadoop cardinality column size " + columns.size() + " is not equal metadata column size " + cols.length + ". Table " + tableName);
+        }
+        Iterator<String> it = columns.iterator();
+        while (it.hasNext()) {
+            String string = (String) it.next();
+            String[] ss = StringUtils.split(string, "\t");
+
+            if (ss.length != 2) {
+                logger.error("The hadoop cardinality value is not valid " + string);
+                continue;
+            }
+            cardi.append(ss[1]);
+            cardi.append(",");
+        }
+        String scardi = cardi.toString();
+        scardi = scardi.substring(0, scardi.length() - 1);
+        tableExd.put(MetadataConstances.TABLE_EXD_CARDINALITY, scardi);
+
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            JsonUtil.writeValueIndent(bos, tableExd);
+            System.out.println(bos.toString());
+            ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
+            String xPath = ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + tableName.toUpperCase() + "." + HiveSourceTableLoader.OUTPUT_SURFIX;
+            writeResource(bis, KylinConfig.getInstanceFromEnv(), xPath);
+        } catch (JsonGenerationException e) {
+            e.printStackTrace();
+        } catch (JsonMappingException e) {
+            e.printStackTrace();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        getMetadataManager().reload();
+    }
+
+    private static void writeResource(InputStream source, KylinConfig dstConfig, String path) throws IOException {
+        ResourceStore store = ResourceStore.getStore(dstConfig);
+        store.putResource(path, source, System.currentTimeMillis());
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION')  or hasPermission(#cube, 'MANAGEMENT')")
+    public void updateCubeNotifyList(CubeInstance cube, List<String> notifyList) throws IOException, CubeIntegrityException {
+        CubeDesc desc = cube.getDescriptor();
+        desc.setNotifyList(notifyList);
+        getMetadataManager().updateCubeDesc(desc);
+    }
+
+    public CubeInstance rebuildLookupSnapshot(String cubeName, String segmentName, String lookupTable) throws IOException {
+        CubeManager cubeMgr = getCubeManager();
+        CubeInstance cube = cubeMgr.getCube(cubeName);
+        CubeSegment seg = cube.getSegment(segmentName, CubeSegmentStatusEnum.READY);
+        cubeMgr.buildSnapshotTable(seg, lookupTable);
+
+        return cube;
+    }
+
+    /**
+     * purge the cube
+     *
+     * @throws IOException
+     * @throws JobException
+     * @throws UnknownHostException
+     * @throws CubeIntegrityException
+     */
+    private void releaseAllSegments(CubeInstance cube) throws IOException, JobException, UnknownHostException, CubeIntegrityException {
+        for (JobInstance jobInstance : this.getJobManager().listJobs(cube.getName(), null)) {
+            if (jobInstance.getStatus() != JobStatusEnum.FINISHED && jobInstance.getStatus() != JobStatusEnum.DISCARDED) {
+                for (JobStep jobStep : jobInstance.getSteps()) {
+                    if (jobStep.getStatus() != JobStepStatusEnum.FINISHED) {
+                        jobStep.setStatus(JobStepStatusEnum.DISCARDED);
+                    }
+                }
+                JobDAO.getInstance(this.getConfig()).updateJobInstance(jobInstance);
+            }
+        }
+
+        cube.getSegments().clear();
+        CubeManager.getInstance(getConfig()).updateCube(cube);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_MODELER)
+    public String[] reloadHiveTable(String tables) throws IOException {
+        Set<String> loaded = HiveSourceTableLoader.reloadHiveTables(tables.split(","), getConfig());
+        getMetadataManager().reload();
+        return (String[]) loaded.toArray(new String[loaded.size()]);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
+    public void syncTableToProject(String[] tables, String project) throws IOException {
+        getProjectManager().updateTableToProject(tables, project);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/JobService.java b/server/src/main/java/com/kylinolap/rest/service/JobService.java
new file mode 100644
index 0000000..af4f16f
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/JobService.java
@@ -0,0 +1,184 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.cube.CubeBuildTypeEnum;
+import com.kylinolap.cube.CubeInstance;
+import com.kylinolap.cube.CubeSegment;
+import com.kylinolap.cube.exception.CubeIntegrityException;
+import com.kylinolap.job.JobInstance;
+import com.kylinolap.job.constant.JobStatusEnum;
+import com.kylinolap.job.exception.InvalidJobInstanceException;
+import com.kylinolap.job.exception.JobException;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.MetricsRequest;
+import com.kylinolap.rest.response.MetricsResponse;
+
+/**
+ * @author ysong1
+ */
+@Component("jobService")
+public class JobService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(CubeService.class);
+
+    @Autowired
+    private AccessService permissionService;
+
+    public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList, final Integer limitValue, final Integer offsetValue) throws IOException, JobException {
+        Integer limit = (null == limitValue) ? 30 : limitValue;
+        Integer offset = (null == offsetValue) ? 0 : offsetValue;
+        List<JobInstance> jobs = listAllJobs(cubeName, projectName, statusList);
+        Collections.sort(jobs);
+
+        if (jobs.size() <= offset) {
+            return Collections.emptyList();
+        }
+
+        if ((jobs.size() - offset) < limit) {
+            return jobs.subList(offset, jobs.size());
+        }
+
+        return jobs.subList(offset, offset + limit);
+    }
+
+    public List<JobInstance> listAllJobs(String cubeName, String projectName, List<JobStatusEnum> statusList) throws IOException, JobException {
+        List<JobInstance> jobs = new ArrayList<JobInstance>();
+        jobs.addAll(this.getJobManager().listJobs(cubeName, projectName));
+
+        if (null == jobs || jobs.size() == 0) {
+            return jobs;
+        }
+
+        List<JobInstance> results = new ArrayList<JobInstance>();
+
+        for (JobInstance job : jobs) {
+            if (null != statusList && statusList.size() > 0) {
+                for (JobStatusEnum status : statusList) {
+                    if (job.getStatus() == status) {
+                        results.add(job);
+                    }
+                }
+            } else {
+                results.add(job);
+            }
+        }
+
+        return results;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public String submitJob(CubeInstance cube, long startDate, long endDate, CubeBuildTypeEnum buildType,String submitter) throws IOException, JobException, InvalidJobInstanceException {
+
+        List<JobInstance> jobInstances = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobInstances) {
+            if (jobInstance.getStatus() == JobStatusEnum.PENDING || jobInstance.getStatus() == JobStatusEnum.RUNNING) {
+                throw new JobException("The cube " + cube.getName() + " has running job(" + jobInstance.getUuid() + ") please discard it and try again.");
+            }
+        }
+
+        String uuid = null;
+        try {
+            List<CubeSegment> cubeSegments = this.getCubeManager().allocateSegments(cube, buildType, startDate, endDate);
+            List<JobInstance> jobs = Lists.newArrayListWithExpectedSize(cubeSegments.size());
+            for (CubeSegment segment : cubeSegments) {
+                uuid = segment.getUuid();
+                JobInstance job = this.getJobManager().createJob(cube.getName(), segment.getName(), segment.getUuid(), buildType,submitter);
+                segment.setLastBuildJobID(uuid);
+                jobs.add(job);
+            }
+            getCubeManager().updateCube(cube);
+            for (JobInstance job: jobs) {
+                this.getJobManager().submitJob(job);
+                permissionService.init(job, null);
+                permissionService.inherit(job, cube);
+            }
+        } catch (CubeIntegrityException e) {
+            throw new InternalErrorException(e.getLocalizedMessage(), e);
+        }
+
+        return uuid;
+    }
+
+    public JobInstance getJobInstance(String uuid) throws IOException, JobException {
+        return this.getJobManager().getJob(uuid);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#job, 'ADMINISTRATION') or hasPermission(#job, 'OPERATION') or hasPermission(#job, 'MANAGEMENT')")
+    public void resumeJob(JobInstance job) throws IOException, JobException {
+        this.getJobManager().resumeJob(job.getUuid());
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#job, 'ADMINISTRATION') or hasPermission(#job, 'OPERATION') or hasPermission(#job, 'MANAGEMENT')")
+    public void cancelJob(JobInstance job) throws IOException, JobException, CubeIntegrityException {
+        CubeInstance cube = this.getCubeManager().getCube(job.getRelatedCube());
+        List<JobInstance> jobs = this.getJobManager().listJobs(cube.getName(), null);
+        for (JobInstance jobInstance : jobs) {
+            if (jobInstance.getStatus() != JobStatusEnum.DISCARDED && jobInstance.getStatus() != JobStatusEnum.FINISHED) {
+                this.getJobManager().discardJob(jobInstance.getUuid());
+            }
+        }
+    }
+
+    public MetricsResponse calculateMetrics(MetricsRequest request) {
+        List<JobInstance> jobs = new ArrayList<JobInstance>();
+
+        try {
+            jobs.addAll(getJobManager().listJobs(null, null));
+        } catch (IOException e) {
+            logger.error("", e);
+        } catch (JobException e) {
+            logger.error("", e);
+        }
+
+        MetricsResponse metrics = new MetricsResponse();
+        int successCount = 0;
+        long totalTime = 0;
+        Date startTime = (null == request.getStartTime()) ? new Date(-1) : request.getStartTime();
+        Date endTime = (null == request.getEndTime()) ? new Date() : request.getEndTime();
+
+        for (JobInstance job : jobs) {
+            if (job.getExecStartTime() > startTime.getTime() && job.getExecStartTime() < endTime.getTime()) {
+                metrics.increase("total");
+                metrics.increase(job.getStatus().name());
+
+                if (job.getStatus() == JobStatusEnum.FINISHED) {
+                    successCount++;
+                    totalTime += (job.getExecEndTime() - job.getExecStartTime());
+                }
+            }
+        }
+
+        metrics.increase("aveExecTime", ((successCount == 0) ? 0 : totalTime / (float) successCount));
+
+        return metrics;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/MetricsService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/MetricsService.java b/server/src/main/java/com/kylinolap/rest/service/MetricsService.java
new file mode 100644
index 0000000..9d8dc4e
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/MetricsService.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import org.springframework.beans.factory.InitializingBean;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Qualifier;
+import org.springframework.stereotype.Component;
+
+import com.codahale.metrics.MetricRegistry;
+import com.kylinolap.job.JobManager;
+import com.kylinolap.rest.metrics.JobMetrics;
+import com.kylinolap.rest.metrics.QueryMetrics;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("metricsService")
+public class MetricsService implements InitializingBean {
+
+    @Autowired
+    @Qualifier("metrics")
+    private MetricRegistry metricRegistry;
+
+    public void registerJobMetrics(final JobManager jobManager) {
+        JobMetrics jobMetrics = JobMetrics.getInstance();
+        jobMetrics.setJobManager(jobManager);
+        metricRegistry.register("JobMetrics", jobMetrics);
+    }
+
+    public void registerQueryMetrics() {
+        metricRegistry.register("QueryMetrics", QueryMetrics.getInstance());
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.springframework.beans.factory.InitializingBean#afterPropertiesSet()
+     */
+    @Override
+    public void afterPropertiesSet() throws Exception {
+        registerQueryMetrics();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4b631f92/server/src/main/java/com/kylinolap/rest/service/ProjectService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/com/kylinolap/rest/service/ProjectService.java b/server/src/main/java/com/kylinolap/rest/service/ProjectService.java
new file mode 100644
index 0000000..73c7302
--- /dev/null
+++ b/server/src/main/java/com/kylinolap/rest/service/ProjectService.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2013-2014 eBay Software Foundation
+ *
+ * Licensed 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 com.kylinolap.rest.service;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.security.core.context.SecurityContextHolder;
+import org.springframework.stereotype.Component;
+
+import com.kylinolap.cube.project.ProjectInstance;
+import com.kylinolap.rest.constant.Constant;
+import com.kylinolap.rest.exception.InternalErrorException;
+import com.kylinolap.rest.request.CreateProjectRequest;
+import com.kylinolap.rest.request.UpdateProjectRequest;
+import com.kylinolap.rest.security.AclPermission;
+
+/**
+ * @author xduo
+ * 
+ */
+@Component("projectService")
+public class ProjectService extends BasicService {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProjectService.class);
+
+    @Autowired
+    private AccessService accessService;
+
+    public ProjectInstance createProject(CreateProjectRequest projectRequest) throws IOException {
+        String projectName = projectRequest.getName();
+        String description = projectRequest.getDescription();
+        ProjectInstance currentProject = getProjectManager().getProject(projectName);
+
+        if (currentProject != null) {
+            throw new InternalErrorException("The project named " + projectName + " already exists");
+        }
+        String owner = SecurityContextHolder.getContext().getAuthentication().getName();
+        ProjectInstance createdProject = getProjectManager().createProject(projectName, owner, description);
+        accessService.init(createdProject, AclPermission.ADMINISTRATION);
+        logger.debug("New project created.");
+
+        return createdProject;
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public ProjectInstance updateProject(UpdateProjectRequest projectRequest) throws IOException {
+        String formerProjectName = projectRequest.getFormerProjectName();
+        String newProjectName = projectRequest.getNewProjectName();
+        String newDescription = projectRequest.getNewDescription();
+
+        ProjectInstance currentProject = getProjectManager().getProject(formerProjectName);
+
+        if (currentProject == null) {
+            throw new InternalErrorException("The project named " + formerProjectName + " does not exists");
+        }
+
+        ProjectInstance updatedProject = getProjectManager().updateProject(currentProject, newProjectName, newDescription);
+
+        logger.debug("Project updated.");
+
+        return updatedProject;
+    }
+
+    public List<ProjectInstance> listAllProjects(final Integer limit, final Integer offset) {
+        List<ProjectInstance> projects = getProjectManager().listAllProjects();
+
+        int climit = (null == limit) ? 30 : limit;
+        int coffset = (null == offset) ? 0 : offset;
+
+        if (projects.size() <= coffset) {
+            return Collections.emptyList();
+        }
+
+        if ((projects.size() - coffset) < climit) {
+            return projects.subList(coffset, projects.size());
+        }
+
+        return projects.subList(coffset, coffset + climit);
+    }
+
+    @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+    public void deleteProject(String projectName) throws IOException {
+        ProjectInstance project = getProjectManager().getProject(projectName);
+        getProjectManager().dropProject(projectName);
+
+        accessService.clean(project, true);
+    }
+
+    /**
+     * @param name
+     * @throws IOException
+     */
+    public void reloadProjectCache(String name) throws IOException {
+        ProjectInstance project = this.getProjectManager().getProject(name);
+        this.getProjectManager().loadProjectCache(project, false);
+    }
+
+    /**
+     * @param name
+     */
+    public void removeProjectCache(String name) {
+        ProjectInstance project = this.getProjectManager().getProject(name);
+        this.getProjectManager().removeProjectCache(project);
+    }
+
+}