From f2f487fe2dc868119562303cb8b9a0b630f704cf Mon Sep 17 00:00:00 2001 From: Li Yang Date: Sat, 9 Dec 2017 00:23:51 +0800 Subject: [PATCH] KYLIN-3092 Synchronize w/r operations on entity-caching managers --- .../apache/kylin/cube/CubeDescManager.java | 311 +++-- .../org/apache/kylin/cube/CubeInstance.java | 31 +- .../org/apache/kylin/cube/CubeManager.java | 1079 +++++++++-------- .../cube/cli/CubeSignatureRefresher.java | 2 +- .../org/apache/kylin/cube/model/CubeDesc.java | 29 +- .../kylin/metadata/TableMetadataManager.java | 583 +++++---- .../kylin/metadata/TempStatementManager.java | 157 ++- .../apache/kylin/metadata/acl/TableACL.java | 11 + .../kylin/metadata/acl/TableACLManager.java | 111 +- .../metadata/cachesync/CachedCrudAssist.java | 72 +- .../kylin/metadata/model/DataModelDesc.java | 7 +- .../metadata/model/DataModelManager.java | 269 ++-- .../metadata/model/ExternalFilterDesc.java | 5 + .../kylin/metadata/model/TableDesc.java | 79 +- .../kylin/metadata/model/TableExtDesc.java | 38 +- .../metadata/project/ProjectInstance.java | 2 +- .../metadata/project/ProjectManager.java | 12 +- .../metadata/streaming/StreamingConfig.java | 6 + .../metadata/streaming/StreamingManager.java | 183 +-- .../metadata/TempStatementManagerTest.java | 6 +- .../streaming/StreamingManagerTest.java | 68 ++ .../kylin/storage/hybrid/HybridInstance.java | 37 +- .../kylin/storage/hybrid/HybridManager.java | 133 +- .../kylin/engine/spark/SparkCubing.java | 4 +- .../DEFAULT_SESSION/temp_table1.json | 1 + .../DEFAULT_SESSION/temp_table2.json | 1 + .../kylin/provision/BuildCubeWithStream.java | 2 +- .../rest/controller/ModelController.java | 2 +- .../rest/service/KafkaConfigService.java | 2 +- .../kylin/rest/service/StreamingService.java | 4 +- .../source/kafka/KafkaConfigManager.java | 166 +-- .../source/kafka/config/KafkaConfig.java | 8 +- 32 files changed, 1746 insertions(+), 1675 deletions(-) create mode 100644 core-metadata/src/test/java/org/apache/kylin/metadata/streaming/StreamingManagerTest.java diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java index f7245492f80..a58ba401303 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java @@ -25,9 +25,9 @@ import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; -import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.cube.cuboid.CuboidManager; import org.apache.kylin.cube.model.CubeDesc; import org.apache.kylin.cube.model.validation.CubeMetadataValidator; @@ -36,9 +36,9 @@ import org.apache.kylin.dimension.DimensionEncoding; import org.apache.kylin.dimension.DimensionEncodingFactory; import org.apache.kylin.measure.topn.TopNMeasureType; -import org.apache.kylin.metadata.MetadataConstants; import org.apache.kylin.metadata.cachesync.Broadcaster; import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.apache.kylin.metadata.datatype.DataType; import org.apache.kylin.metadata.model.MeasureDesc; @@ -59,8 +59,6 @@ public class CubeDescManager { private static final Logger logger = LoggerFactory.getLogger(CubeDescManager.class); - public static final Serializer CUBE_DESC_SERIALIZER = new JsonSerializer(CubeDesc.class); - public static CubeDescManager getInstance(KylinConfig config) { return config.getManager(CubeDescManager.class); } @@ -69,20 +67,42 @@ public static CubeDescManager getInstance(KylinConfig config) { static CubeDescManager newInstance(KylinConfig config) throws IOException { return new CubeDescManager(config); } - + // ============================================================================ private KylinConfig config; + // name ==> CubeDesc private CaseInsensitiveStringCache cubeDescMap; + private CachedCrudAssist crud; + + // protects concurrent operations around the cached map, to avoid for example + // writing an entity in the middle of reloading it (dirty read) + private AutoReadWriteLock descMapLock = new AutoReadWriteLock(); - private CubeDescManager(KylinConfig config) throws IOException { - logger.info("Initializing CubeDescManager with config " + config); - this.config = config; + private CubeDescManager(KylinConfig cfg) throws IOException { + logger.info("Initializing CubeDescManager with config " + cfg); + this.config = cfg; this.cubeDescMap = new CaseInsensitiveStringCache(config, "cube_desc"); + this.crud = new CachedCrudAssist(getStore(), ResourceStore.CUBE_DESC_RESOURCE_ROOT, CubeDesc.class, + cubeDescMap) { + @Override + protected CubeDesc initEntityAfterReload(CubeDesc cubeDesc, String resourceName) { + if (cubeDesc.isDraft()) + throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' must not be a draft"); + + try { + cubeDesc.init(config); + } catch (Exception e) { + logger.warn("Broken cube desc " + cubeDesc.resourceName(), e); + cubeDesc.addError(e.getMessage()); + } + return cubeDesc; + } + }; // touch lower level metadata before registering my listener - reloadAllCubeDesc(); + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new CubeDescSyncListener(), "cube_desc"); } @@ -93,7 +113,7 @@ public void onProjectSchemaChange(Broadcaster broadcaster, String project) throw for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) { if (real instanceof CubeInstance) { String descName = ((CubeInstance) real).getDescName(); - reloadCubeDescLocal(descName); + reloadCubeDescQuietly(descName); } } } @@ -108,7 +128,7 @@ public void onEntityChange(Broadcaster broadcaster, String entity, Event event, if (event == Event.DROP) removeLocalCubeDesc(cubeDescName); else - reloadCubeDescLocal(cubeDescName); + reloadCubeDescQuietly(cubeDescName); for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByModel(modelName)) { broadcaster.notifyProjectSchemaUpdate(prj.getName()); @@ -117,58 +137,45 @@ public void onEntityChange(Broadcaster broadcaster, String entity, Event event, } public CubeDesc getCubeDesc(String name) { - return cubeDescMap.get(name); + try (AutoLock lock = descMapLock.lockForRead()) { + return cubeDescMap.get(name); + } } public List listAllDesc() { - return new ArrayList(cubeDescMap.values()); - } - - /** - * Reload CubeDesc from resource store It will be triggered by an desc - * update event. - * - * @param name - * @throws IOException - */ - public CubeDesc reloadCubeDescLocal(String name) throws IOException { - // Broken CubeDesc is not allowed to be saved and broadcast. - CubeDesc ndesc = loadCubeDesc(CubeDesc.concatResourcePath(name), false); - - cubeDescMap.putLocal(ndesc.getName(), ndesc); - clearCuboidCache(ndesc.getName()); - - // if related cube is in DESCBROKEN state before, change it back to DISABLED - CubeManager cubeManager = CubeManager.getInstance(config); - for (CubeInstance cube : cubeManager.getCubesByDesc(name)) { - if (cube.getStatus() == RealizationStatusEnum.DESCBROKEN) { - cubeManager.reloadCubeLocal(cube.getName()); - } + try (AutoLock lock = descMapLock.lockForRead()) { + return new ArrayList(cubeDescMap.values()); } - - return ndesc; } - - private CubeDesc loadCubeDesc(String path, boolean allowBroken) throws IOException { - ResourceStore store = getStore(); - CubeDesc ndesc = store.getResource(path, CubeDesc.class, CUBE_DESC_SERIALIZER); - if (ndesc == null) - throw new IllegalArgumentException("No cube desc found at " + path); - if (ndesc.isDraft()) - throw new IllegalArgumentException("CubeDesc '" + ndesc.getName() + "' must not be a draft"); - - try { - ndesc.init(config); + + public CubeDesc reloadCubeDescQuietly(String name) { + try (AutoLock lock = descMapLock.lockForWrite()) { + return reloadCubeDescLocal(name); } catch (Exception e) { - logger.warn("Broken cube desc " + path, e); - ndesc.addError(e.getMessage()); + logger.error("Failed to reload CubeDesc " + name, e); + return null; } + } - if (!allowBroken && !ndesc.getError().isEmpty()) { - throw new IllegalStateException("Cube desc at " + path + " has issues: " + ndesc.getError()); + public CubeDesc reloadCubeDescLocal(String name) throws IOException { + try (AutoLock lock = descMapLock.lockForWrite()) { + CubeDesc ndesc = crud.reload(name); + clearCuboidCache(name); + + // Broken CubeDesc is not allowed to be saved and broadcast. + if (ndesc.isBroken()) + throw new IllegalStateException("CubeDesc " + name + " is broken"); + + // if related cube is in DESCBROKEN state before, change it back to DISABLED + CubeManager cubeManager = CubeManager.getInstance(config); + for (CubeInstance cube : cubeManager.getCubesByDesc(name)) { + if (cube.getStatus() == RealizationStatusEnum.DESCBROKEN) { + cube.init(config); + } + } + + return ndesc; } - - return ndesc; } /** @@ -179,38 +186,83 @@ private CubeDesc loadCubeDesc(String path, boolean allowBroken) throws IOExcepti * @throws IOException */ public CubeDesc createCubeDesc(CubeDesc cubeDesc) throws IOException { - if (cubeDesc.getUuid() == null || cubeDesc.getName() == null) - throw new IllegalArgumentException(); - if (cubeDescMap.containsKey(cubeDesc.getName())) - throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' already exists"); - if (cubeDesc.isDraft()) - throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' must not be a draft"); - - try { - cubeDesc.init(config); - } catch (Exception e) { - logger.warn("Broken cube desc " + cubeDesc, e); - cubeDesc.addError(e.getMessage()); - } - postProcessCubeDesc(cubeDesc); - // Check base validation - if (!cubeDesc.getError().isEmpty()) { - return cubeDesc; - } - // Semantic validation - CubeMetadataValidator validator = new CubeMetadataValidator(); - ValidateContext context = validator.validate(cubeDesc); - if (!context.ifPass()) { + try (AutoLock lock = descMapLock.lockForWrite()) { + if (cubeDesc.getUuid() == null || cubeDesc.getName() == null) + throw new IllegalArgumentException(); + if (cubeDescMap.containsKey(cubeDesc.getName())) + throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' already exists"); + if (cubeDesc.isDraft()) + throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' must not be a draft"); + + try { + cubeDesc.init(config); + } catch (Exception e) { + logger.warn("Broken cube desc " + cubeDesc, e); + cubeDesc.addError(e.getMessage()); + } + + postProcessCubeDesc(cubeDesc); + // Check base validation + if (!cubeDesc.getError().isEmpty()) { + return cubeDesc; + } + // Semantic validation + CubeMetadataValidator validator = new CubeMetadataValidator(); + ValidateContext context = validator.validate(cubeDesc); + if (!context.ifPass()) { + return cubeDesc; + } + + cubeDesc.setSignature(cubeDesc.calculateSignature()); + + // save resource + crud.save(cubeDesc); + return cubeDesc; } + } + + /** + * Update CubeDesc with the input. Broadcast the event into cluster + * + * @param desc + * @return + * @throws IOException + */ + public CubeDesc updateCubeDesc(CubeDesc desc) throws IOException { + try (AutoLock lock = descMapLock.lockForWrite()) { + // Validate CubeDesc + if (desc.getUuid() == null || desc.getName() == null) + throw new IllegalArgumentException(); + String name = desc.getName(); + if (!cubeDescMap.containsKey(name)) + throw new IllegalArgumentException("CubeDesc '" + name + "' does not exist."); + if (desc.isDraft()) + throw new IllegalArgumentException("CubeDesc '" + desc.getName() + "' must not be a draft"); + + try { + desc.init(config); + } catch (Exception e) { + logger.warn("Broken cube desc " + desc, e); + desc.addError(e.getMessage()); + return desc; + } + + postProcessCubeDesc(desc); + // Semantic validation + CubeMetadataValidator validator = new CubeMetadataValidator(); + ValidateContext context = validator.validate(desc); + if (!context.ifPass()) { + return desc; + } - cubeDesc.setSignature(cubeDesc.calculateSignature()); + desc.setSignature(desc.calculateSignature()); - String path = cubeDesc.getResourcePath(); - getStore().putResource(path, cubeDesc, CUBE_DESC_SERIALIZER); - cubeDescMap.put(cubeDesc.getName(), cubeDesc); + // save resource + crud.save(desc); - return cubeDesc; + return desc; + } } /** @@ -259,16 +311,18 @@ private void postProcessCubeDesc(CubeDesc cubeDesc) { // remove cubeDesc public void removeCubeDesc(CubeDesc cubeDesc) throws IOException { - String path = cubeDesc.getResourcePath(); - getStore().deleteResource(path); - cubeDescMap.remove(cubeDesc.getName()); - clearCuboidCache(cubeDesc.getName()); + try (AutoLock lock = descMapLock.lockForWrite()) { + crud.delete(cubeDesc); + clearCuboidCache(cubeDesc.getName()); + } } // remove cubeDesc public void removeLocalCubeDesc(String name) throws IOException { - cubeDescMap.removeLocal(name); - clearCuboidCache(name); + try (AutoLock lock = descMapLock.lockForWrite()) { + cubeDescMap.removeLocal(name); + clearCuboidCache(name); + } } private void clearCuboidCache(String descName) { @@ -276,87 +330,6 @@ private void clearCuboidCache(String descName) { CuboidManager.getInstance(config).clearCache(descName); } - private void reloadAllCubeDesc() throws IOException { - ResourceStore store = getStore(); - logger.info("Reloading Cube Metadata from folder " - + store.getReadableResourcePath(ResourceStore.CUBE_DESC_RESOURCE_ROOT)); - - cubeDescMap.clear(); - - List paths = store.collectResourceRecursively(ResourceStore.CUBE_DESC_RESOURCE_ROOT, - MetadataConstants.FILE_SURFIX); - for (String path : paths) { - CubeDesc desc = null; - try { - desc = loadCubeDesc(path, true); - } catch (Exception e) { - logger.error("Error during load cube desc, skipping " + path, e); - continue; - } - - if (!path.equals(desc.getResourcePath())) { - logger.error( - "Skip suspicious desc at " + path + ", " + desc + " should be at " + desc.getResourcePath()); - continue; - } - if (cubeDescMap.containsKey(desc.getName())) { - logger.error("Dup CubeDesc name '" + desc.getName() + "' on path " + path); - continue; - } - - cubeDescMap.putLocal(desc.getName(), desc); - } - - logger.info("Loaded " + cubeDescMap.size() + " Cube Desc(s)"); - } - - /** - * Update CubeDesc with the input. Broadcast the event into cluster - * - * @param desc - * @return - * @throws IOException - */ - public CubeDesc updateCubeDesc(CubeDesc desc) throws IOException { - // Validate CubeDesc - if (desc.getUuid() == null || desc.getName() == null) - throw new IllegalArgumentException(); - String name = desc.getName(); - if (!cubeDescMap.containsKey(name)) - throw new IllegalArgumentException("CubeDesc '" + name + "' does not exist."); - if (desc.isDraft()) - throw new IllegalArgumentException("CubeDesc '" + desc.getName() + "' must not be a draft"); - - try { - desc.init(config); - } catch (Exception e) { - logger.warn("Broken cube desc " + desc, e); - desc.addError(e.getMessage()); - return desc; - } - - postProcessCubeDesc(desc); - // Semantic validation - CubeMetadataValidator validator = new CubeMetadataValidator(); - ValidateContext context = validator.validate(desc); - if (!context.ifPass()) { - return desc; - } - - desc.setSignature(desc.calculateSignature()); - - // Save Source - String path = desc.getResourcePath(); - getStore().putResource(path, desc, CUBE_DESC_SERIALIZER); - - // Reload the CubeDesc - CubeDesc ndesc = loadCubeDesc(path, false); - // Here replace the old one - cubeDescMap.put(ndesc.getName(), desc); - - return ndesc; - } - private ResourceStore getStore() { return ResourceStore.getStore(this.config); } diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java index d1c5166d6e9..1be7923150a 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java @@ -18,6 +18,8 @@ package org.apache.kylin.cube; +import static com.google.common.base.Preconditions.checkNotNull; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -50,6 +52,8 @@ import org.apache.kylin.metadata.realization.RealizationStatusEnum; import org.apache.kylin.metadata.realization.RealizationType; import org.apache.kylin.metadata.realization.SQLDigest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; @@ -63,6 +67,8 @@ @SuppressWarnings("serial") @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) public class CubeInstance extends RootPersistentEntity implements IRealization, IBuildable { + private static final Logger logger = LoggerFactory.getLogger(CubeInstance.class); + public static final int COST_WEIGHT_MEASURE = 1; public static final int COST_WEIGHT_DIMENSION = 10; public static final int COST_WEIGHT_INNER_JOIN = 100; @@ -121,6 +127,24 @@ public static CubeInstance create(String cubeName, CubeDesc cubeDesc) { // default constructor for jackson public CubeInstance() { } + + void init(KylinConfig config) { + CubeDesc cubeDesc = CubeDescManager.getInstance(config).getCubeDesc(descName); + checkNotNull(cubeDesc, "cube descriptor '%s' (for cube '%s') not found", descName, name); + + if (cubeDesc.isBroken()) { + setStatus(RealizationStatusEnum.DESCBROKEN); + logger.error("cube descriptor {} (for cube '{}') is broken", cubeDesc.getResourcePath(), name); + for (String error : cubeDesc.getError()) { + logger.error("Error: {}", error); + } + } else if (getStatus() == RealizationStatusEnum.DESCBROKEN) { + setStatus(RealizationStatusEnum.DISABLED); + logger.info("cube {} changed from DESCBROKEN to DISABLED", name); + } + + setConfig((KylinConfigExt) cubeDesc.getConfig()); + } public CuboidScheduler getCuboidScheduler() { if (cuboidScheduler != null) @@ -174,9 +198,14 @@ public boolean allowBrokenDescriptor() { return (getStatus() == RealizationStatusEnum.DISABLED || getStatus() == RealizationStatusEnum.DESCBROKEN) && segments.isEmpty(); } + + @Override + public String resourceName() { + return name; + } public String getResourcePath() { - return concatResourcePath(name); + return concatResourcePath(resourceName()); } public static String concatResourcePath(String cubeName) { diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java index e00735c73b8..3220a0f0785 100755 --- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java @@ -18,13 +18,9 @@ package org.apache.kylin.cube; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -37,10 +33,11 @@ import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.KylinConfigExt; import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.common.util.Dictionary; import org.apache.kylin.common.util.Pair; import org.apache.kylin.cube.cuboid.Cuboid; @@ -54,6 +51,7 @@ import org.apache.kylin.metadata.TableMetadataManager; import org.apache.kylin.metadata.cachesync.Broadcaster; import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.apache.kylin.metadata.model.DataModelDesc; import org.apache.kylin.metadata.model.JoinDesc; @@ -87,7 +85,7 @@ public class CubeManager implements IRealizationProvider { private static String ALPHA_NUM = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ"; private static int HBASE_TABLE_LENGTH = 10; - public static final Serializer CUBE_SERIALIZER = new JsonSerializer(CubeInstance.class); + public static final Serializer CUBE_SERIALIZER = new JsonSerializer<>(CubeInstance.class); private static final Logger logger = LoggerFactory.getLogger(CubeManager.class); @@ -103,21 +101,41 @@ static CubeManager newInstance(KylinConfig config) throws IOException { // ============================================================================ private KylinConfig config; + // cube name ==> CubeInstance private CaseInsensitiveStringCache cubeMap; - // "table/column" ==> lookup table - // private SingleValueCache lookupTables = new SingleValueCache(Broadcaster.TYPE.METADATA); + private CachedCrudAssist crud; + + // protects concurrent operations around the cached map, to avoid for example + // writing an entity in the middle of reloading it (dirty read) + private AutoReadWriteLock cubeMapLock = new AutoReadWriteLock(); // for generation hbase table name of a new segment private ConcurrentMap usedStorageLocation = new ConcurrentHashMap<>(); - private CubeManager(KylinConfig config) throws IOException { + // a few inner classes to group related methods + private SegmentAssist segAssist = new SegmentAssist(); + private DictionaryAssist dictAssist = new DictionaryAssist(); + + private CubeManager(KylinConfig cfg) throws IOException { logger.info("Initializing CubeManager with config " + config); - this.config = config; + this.config = cfg; this.cubeMap = new CaseInsensitiveStringCache(config, "cube"); + this.crud = new CachedCrudAssist(getStore(), ResourceStore.CUBE_RESOURCE_ROOT, CubeInstance.class, + cubeMap) { + @Override + protected CubeInstance initEntityAfterReload(CubeInstance cube, String resourceName) { + cube.init(config); + + for (CubeSegment segment : cube.getSegments()) { + usedStorageLocation.put(segment.getUuid(), segment.getStorageLocationIdentifier()); + } + return cube; + } + }; // touch lower level metadata before registering my listener - loadAllCubeInstance(); + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new CubeSyncListener(), "cube"); } @@ -127,7 +145,7 @@ private class CubeSyncListener extends Broadcaster.Listener { public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException { for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) { if (real instanceof CubeInstance) { - reloadCubeLocal(real.getName()); + reloadCubeQuietly(real.getName()); } } } @@ -140,7 +158,7 @@ public void onEntityChange(Broadcaster broadcaster, String entity, Event event, if (event == Event.DROP) removeCubeLocal(cubeName); else - reloadCubeLocal(cubeName); + reloadCubeQuietly(cubeName); for (ProjectInstance prj : ProjectManager.getInstance(config).findProjects(RealizationType.CUBE, cubeName)) { @@ -150,20 +168,25 @@ public void onEntityChange(Broadcaster broadcaster, String entity, Event event, } public List listAllCubes() { - return new ArrayList(cubeMap.values()); + try (AutoLock lock = cubeMapLock.lockForRead()) { + return new ArrayList(cubeMap.values()); + } } public CubeInstance getCube(String cubeName) { - return cubeMap.get(cubeName); + try (AutoLock lock = cubeMapLock.lockForRead()) { + return cubeMap.get(cubeName); + } } public CubeInstance getCubeByUuid(String uuid) { - Collection copy = new ArrayList(cubeMap.values()); - for (CubeInstance cube : copy) { - if (uuid.equals(cube.getUuid())) - return cube; + try (AutoLock lock = cubeMapLock.lockForRead()) { + for (CubeInstance cube : cubeMap.values()) { + if (uuid.equals(cube.getUuid())) + return cube; + } + return null; } - return null; } /** @@ -174,148 +197,57 @@ public CubeInstance getCubeByUuid(String uuid) { * @return */ public List getCubesByDesc(String descName) { - - List list = listAllCubes(); - List result = new ArrayList(); - Iterator it = list.iterator(); - while (it.hasNext()) { - CubeInstance ci = it.next(); - if (descName.equalsIgnoreCase(ci.getDescName())) { - result.add(ci); + try (AutoLock lock = cubeMapLock.lockForRead()) { + List list = listAllCubes(); + List result = new ArrayList(); + Iterator it = list.iterator(); + while (it.hasNext()) { + CubeInstance ci = it.next(); + if (descName.equalsIgnoreCase(ci.getDescName())) { + result.add(ci); + } } + return result; } - return result; - } - - public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable) - throws IOException { - CubeDesc cubeDesc = cubeSeg.getCubeDesc(); - if (!cubeDesc.getAllColumnsNeedDictionaryBuilt().contains(col)) - return null; - - String builderClass = cubeDesc.getDictionaryBuilderClass(col); - DictionaryInfo dictInfo = getDictionaryManager().buildDictionary(col, inpTable, builderClass); - - saveDictionaryInfo(cubeSeg, col, dictInfo); - return dictInfo; } - public DictionaryInfo saveDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable, - Dictionary dict) throws IOException { - CubeDesc cubeDesc = cubeSeg.getCubeDesc(); - if (!cubeDesc.getAllColumnsNeedDictionaryBuilt().contains(col)) - return null; - - DictionaryInfo dictInfo = getDictionaryManager().saveDictionary(col, inpTable, dict); - - saveDictionaryInfo(cubeSeg, col, dictInfo); - return dictInfo; - } - - private void saveDictionaryInfo(CubeSegment cubeSeg, TblColRef col, DictionaryInfo dictInfo) throws IOException { - if (dictInfo != null) { - Dictionary dict = dictInfo.getDictionaryObject(); - cubeSeg.putDictResPath(col, dictInfo.getResourcePath()); - cubeSeg.getRowkeyStats().add(new Object[] { col.getIdentity(), dict.getSize(), dict.getSizeOfId() }); - - CubeUpdate update = new CubeUpdate(cubeSeg.getCubeInstance()); - update.setToUpdateSegs(cubeSeg); - updateCube(update); - } - } - - /** - * return null if no dictionary for given column - */ - @SuppressWarnings("unchecked") - public Dictionary getDictionary(CubeSegment cubeSeg, TblColRef col) { - DictionaryInfo info = null; - try { - DictionaryManager dictMgr = getDictionaryManager(); - String dictResPath = cubeSeg.getDictResPath(col); - if (dictResPath == null) - return null; - - info = dictMgr.getDictionaryInfo(dictResPath); - if (info == null) - throw new IllegalStateException("No dictionary found by " + dictResPath - + ", invalid cube state; cube segment" + cubeSeg + ", col " + col); - } catch (IOException e) { - throw new IllegalStateException("Failed to get dictionary for cube segment" + cubeSeg + ", col" + col, e); - } - return (Dictionary) info.getDictionaryObject(); - } - - public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable) throws IOException { - TableMetadataManager metaMgr = getTableManager(); - SnapshotManager snapshotMgr = getSnapshotManager(); - - TableDesc tableDesc = new TableDesc(metaMgr.getTableDesc(lookupTable, cubeSeg.getProject())); - IReadableTable hiveTable = SourceFactory.createReadableTable(tableDesc); - SnapshotTable snapshot = snapshotMgr.buildSnapshot(hiveTable, tableDesc); - - cubeSeg.putSnapshotResPath(lookupTable, snapshot.getResourcePath()); - CubeUpdate cubeBuilder = new CubeUpdate(cubeSeg.getCubeInstance()); - cubeBuilder.setToUpdateSegs(cubeSeg); - updateCube(cubeBuilder); - - return snapshot; - } - - // sync on update - public CubeInstance dropCube(String cubeName, boolean deleteDesc) throws IOException { - logger.info("Dropping cube '" + cubeName + "'"); - // load projects before remove cube from project - - // delete cube instance and cube desc - CubeInstance cube = getCube(cubeName); - - // remove cube and update cache - getStore().deleteResource(cube.getResourcePath()); - cubeMap.remove(cube.getName()); - Cuboid.clearCache(cube); - - if (deleteDesc && cube.getDescriptor() != null) { - CubeDescManager.getInstance(config).removeCubeDesc(cube.getDescriptor()); - } - - // delete cube from project - ProjectManager.getInstance(config).removeRealizationsFromProjects(RealizationType.CUBE, cubeName); - - return cube; - } - - // sync on update public CubeInstance createCube(String cubeName, String projectName, CubeDesc desc, String owner) throws IOException { - logger.info("Creating cube '" + projectName + "-->" + cubeName + "' from desc '" + desc.getName() + "'"); + try (AutoLock lock = cubeMapLock.lockForWrite()) { + logger.info("Creating cube '" + projectName + "-->" + cubeName + "' from desc '" + desc.getName() + "'"); - // save cube resource - CubeInstance cube = CubeInstance.create(cubeName, desc); - cube.setOwner(owner); - updateCubeWithRetry(new CubeUpdate(cube), 0); + // save cube resource + CubeInstance cube = CubeInstance.create(cubeName, desc); + cube.setOwner(owner); + updateCubeWithRetry(new CubeUpdate(cube), 0); - ProjectManager.getInstance(config).moveRealizationToProject(RealizationType.CUBE, cubeName, projectName, owner); + ProjectManager.getInstance(config).moveRealizationToProject(RealizationType.CUBE, cubeName, projectName, + owner); - return cube; + return cube; + } } public CubeInstance createCube(CubeInstance cube, String projectName, String owner) throws IOException { - logger.info("Creating cube '" + projectName + "-->" + cube.getName() + "' from instance object. '"); + try (AutoLock lock = cubeMapLock.lockForWrite()) { + logger.info("Creating cube '" + projectName + "-->" + cube.getName() + "' from instance object. '"); - // save cube resource - cube.setOwner(owner); - updateCubeWithRetry(new CubeUpdate(cube), 0); + // save cube resource + cube.setOwner(owner); + updateCubeWithRetry(new CubeUpdate(cube), 0); - ProjectManager.getInstance(config).moveRealizationToProject(RealizationType.CUBE, cube.getName(), projectName, - owner); + ProjectManager.getInstance(config).moveRealizationToProject(RealizationType.CUBE, cube.getName(), + projectName, owner); - return cube; + return cube; + } } public CubeInstance updateCube(CubeUpdate update) throws IOException { - CubeInstance cube = updateCubeWithRetry(update, 0); - return cube; + try (AutoLock lock = cubeMapLock.lockForWrite()) { + CubeInstance cube = updateCubeWithRetry(update, 0); + return cube; + } } private CubeInstance updateCubeWithRetry(CubeUpdate update, int retry) throws IOException { @@ -378,7 +310,7 @@ private CubeInstance updateCubeWithRetry(CubeUpdate update, int retry) throws IO } try { - getStore().putResource(cube.getResourcePath(), cube, CUBE_SERIALIZER); + crud.save(cube); } catch (IllegalStateException ise) { logger.warn("Write conflict to update cube " + cube.getName() + " at try " + retry + ", will retry..."); if (retry >= 7) { @@ -386,7 +318,7 @@ private CubeInstance updateCubeWithRetry(CubeUpdate update, int retry) throws IO throw ise; } - cube = reloadCubeLocal(cube.getName()); + cube = crud.reload(cube.getName()); update.setCubeInstance(cube); retry++; cube = updateCubeWithRetry(update, retry); @@ -402,486 +334,607 @@ private CubeInstance updateCubeWithRetry(CubeUpdate update, int retry) throws IO } } - cubeMap.put(cube.getName(), cube); - //this is a duplicate call to take care of scenarios where REST cache service unavailable ProjectManager.getInstance(cube.getConfig()).clearL2Cache(); return cube; } - // append a full build segment - public CubeSegment appendSegment(CubeInstance cube) throws IOException { - return appendSegment(cube, null, null, null, null); + public CubeInstance reloadCubeQuietly(String cubeName) { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + CubeInstance cube = crud.reloadQuietly(cubeName); + if (cube != null) + Cuboid.clearCache(cube); + return cube; + } } - public CubeSegment appendSegment(CubeInstance cube, TSRange tsRange) throws IOException { - return appendSegment(cube, tsRange, null, null, null); + public void removeCubeLocal(String cubeName) { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + CubeInstance cube = cubeMap.get(cubeName); + if (cube != null) { + cubeMap.removeLocal(cubeName); + for (CubeSegment segment : cube.getSegments()) { + usedStorageLocation.remove(segment.getUuid()); + } + Cuboid.clearCache(cube); + } + } } - public CubeSegment appendSegment(CubeInstance cube, SourcePartition src) throws IOException { - return appendSegment(cube, src.getTSRange(), src.getSegRange(), src.getSourcePartitionOffsetStart(), - src.getSourcePartitionOffsetEnd()); - } + public CubeInstance dropCube(String cubeName, boolean deleteDesc) throws IOException { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + logger.info("Dropping cube '" + cubeName + "'"); + // load projects before remove cube from project - CubeSegment appendSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange, - Map sourcePartitionOffsetStart, Map sourcePartitionOffsetEnd) - throws IOException { - checkInputRanges(tsRange, segRange); - checkBuildingSegment(cube); + // delete cube instance and cube desc + CubeInstance cube = getCube(cubeName); + + // remove cube and update cache + crud.delete(cube); + Cuboid.clearCache(cube); - // fix start/end a bit - if (cube.getModel().getPartitionDesc().isPartitioned()) { - // if missing start, set it to where last time ends - CubeSegment last = cube.getLastSegment(); - if (last != null && !last.isOffsetCube() && tsRange.start.v == 0) { - tsRange = new TSRange(last.getTSRange().end.v, tsRange.end.v); + if (deleteDesc && cube.getDescriptor() != null) { + CubeDescManager.getInstance(config).removeCubeDesc(cube.getDescriptor()); } - } else { - // full build - tsRange = null; - segRange = null; - } - CubeSegment newSegment = newSegment(cube, tsRange, segRange); - newSegment.setSourcePartitionOffsetStart(sourcePartitionOffsetStart); - newSegment.setSourcePartitionOffsetEnd(sourcePartitionOffsetEnd); - validateNewSegments(cube, newSegment); + // delete cube from project + ProjectManager.getInstance(config).removeRealizationsFromProjects(RealizationType.CUBE, cubeName); - CubeUpdate cubeBuilder = new CubeUpdate(cube); - cubeBuilder.setToAddSegs(newSegment); - updateCube(cubeBuilder); - return newSegment; + return cube; + } } - public CubeSegment refreshSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange) throws IOException { - checkInputRanges(tsRange, segRange); - checkBuildingSegment(cube); - - if (cube.getModel().getPartitionDesc().isPartitioned() == false) { - // full build - tsRange = null; - segRange = null; - } + @VisibleForTesting + /*private*/ String generateStorageLocation() { + String namePrefix = config.getHBaseTableNamePrefix(); + String tableName = ""; + Random ran = new Random(); + do { + StringBuffer sb = new StringBuffer(); + sb.append(namePrefix); + for (int i = 0; i < HBASE_TABLE_LENGTH; i++) { + sb.append(ALPHA_NUM.charAt(ran.nextInt(ALPHA_NUM.length()))); + } + tableName = sb.toString(); + } while (this.usedStorageLocation.containsValue(tableName)); + return tableName; + } - CubeSegment newSegment = newSegment(cube, tsRange, segRange); + private boolean isReady(CubeSegment seg) { + return seg.getStatus() == SegmentStatusEnum.READY; + } - Pair pair = cube.getSegments().fitInSegments(newSegment); - if (pair.getFirst() == false || pair.getSecond() == false) - throw new IllegalArgumentException("The new refreshing segment " + newSegment - + " does not match any existing segment in cube " + cube); + private TableMetadataManager getTableManager() { + return TableMetadataManager.getInstance(config); + } - if (segRange != null) { - CubeSegment toRefreshSeg = null; - for (CubeSegment cubeSegment : cube.getSegments()) { - if (cubeSegment.getSegRange().equals(segRange)) { - toRefreshSeg = cubeSegment; - break; - } - } + private DictionaryManager getDictionaryManager() { + return DictionaryManager.getInstance(config); + } - if (toRefreshSeg == null) { - throw new IllegalArgumentException("For streaming cube, only one segment can be refreshed at one time"); - } + private SnapshotManager getSnapshotManager() { + return SnapshotManager.getInstance(config); + } - newSegment.setSourcePartitionOffsetStart(toRefreshSeg.getSourcePartitionOffsetStart()); - newSegment.setSourcePartitionOffsetEnd(toRefreshSeg.getSourcePartitionOffsetEnd()); - } + private ResourceStore getStore() { + return ResourceStore.getStore(this.config); + } - CubeUpdate cubeBuilder = new CubeUpdate(cube); - cubeBuilder.setToAddSegs(newSegment); - updateCube(cubeBuilder); + @Override + public RealizationType getRealizationType() { + return RealizationType.CUBE; + } - return newSegment; + @Override + public IRealization getRealization(String name) { + return getCube(name); } - public CubeSegment mergeSegments(CubeInstance cube, TSRange tsRange, SegmentRange segRange, boolean force) - throws IOException { - if (cube.getSegments().isEmpty()) - throw new IllegalArgumentException("Cube " + cube + " has no segments"); - - checkInputRanges(tsRange, segRange); - checkBuildingSegment(cube); - checkCubeIsPartitioned(cube); - - if (cube.getSegments().getFirstSegment().isOffsetCube()) { - // offset cube, merge by date range? - if (segRange == null && tsRange != null) { - Pair pair = cube.getSegments(SegmentStatusEnum.READY) - .findMergeOffsetsByDateRange(tsRange, Long.MAX_VALUE); - if (pair == null) - throw new IllegalArgumentException("Find no segments to merge by " + tsRange + " for cube " + cube); - segRange = new SegmentRange(pair.getFirst().getSegRange().start, pair.getSecond().getSegRange().end); - } - tsRange = null; - Preconditions.checkArgument(segRange != null); - } else { - segRange = null; - Preconditions.checkArgument(tsRange != null); - } - - CubeSegment newSegment = newSegment(cube, tsRange, segRange); - - Segments mergingSegments = cube.getMergingSegments(newSegment); - if (mergingSegments.size() <= 1) - throw new IllegalArgumentException("Range " + newSegment.getSegRange() - + " must contain at least 2 segments, but there is " + mergingSegments.size()); - - CubeSegment first = mergingSegments.get(0); - CubeSegment last = mergingSegments.get(mergingSegments.size() - 1); - if (force == false) { - for (int i = 0; i < mergingSegments.size() - 1; i++) { - if (!mergingSegments.get(i).getSegRange().connects(mergingSegments.get(i + 1).getSegRange())) - throw new IllegalStateException("Merging segments must not have gaps between " - + mergingSegments.get(i) + " and " + mergingSegments.get(i + 1)); - } - } - if (first.isOffsetCube()) { - newSegment.setSegRange(new SegmentRange(first.getSegRange().start, last.getSegRange().end)); - newSegment.setSourcePartitionOffsetStart(first.getSourcePartitionOffsetStart()); - newSegment.setSourcePartitionOffsetEnd(last.getSourcePartitionOffsetEnd()); - newSegment.setTSRange(null); - } else { - newSegment.setTSRange(new TSRange(mergingSegments.getTSStart(), mergingSegments.getTSEnd())); - newSegment.setSegRange(null); - } - - if (force == false) { - List emptySegment = Lists.newArrayList(); - for (CubeSegment seg : mergingSegments) { - if (seg.getSizeKB() == 0) { - emptySegment.add(seg.getName()); - } - } + // ============================================================================ + // Segment related methods + // ============================================================================ - if (emptySegment.size() > 0) { - throw new IllegalArgumentException( - "Empty cube segment found, couldn't merge unless 'forceMergeEmptySegment' set to true: " - + emptySegment); - } - } + // append a full build segment + public CubeSegment appendSegment(CubeInstance cube) throws IOException { + return appendSegment(cube, null, null, null, null); + } - validateNewSegments(cube, newSegment); + public CubeSegment appendSegment(CubeInstance cube, TSRange tsRange) throws IOException { + return appendSegment(cube, tsRange, null, null, null); + } - CubeUpdate cubeBuilder = new CubeUpdate(cube); - cubeBuilder.setToAddSegs(newSegment); - updateCube(cubeBuilder); + public CubeSegment appendSegment(CubeInstance cube, SourcePartition src) throws IOException { + return appendSegment(cube, src.getTSRange(), src.getSegRange(), src.getSourcePartitionOffsetStart(), + src.getSourcePartitionOffsetEnd()); + } - return newSegment; + CubeSegment appendSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange, + Map sourcePartitionOffsetStart, Map sourcePartitionOffsetEnd) + throws IOException { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + return segAssist.appendSegment(cube, tsRange, segRange, sourcePartitionOffsetStart, + sourcePartitionOffsetEnd); + } } - private void checkInputRanges(TSRange tsRange, SegmentRange segRange) { - if (tsRange != null && segRange != null) { - throw new IllegalArgumentException( - "Build or refresh cube segment either by TSRange or by SegmentRange, not both."); + public CubeSegment refreshSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange) throws IOException { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + return segAssist.refreshSegment(cube, tsRange, segRange); } } - private void checkBuildingSegment(CubeInstance cube) { - int maxBuldingSeg = cube.getConfig().getMaxBuildingSegments(); - if (cube.getBuildingSegments().size() >= maxBuldingSeg) { - throw new IllegalStateException( - "There is already " + cube.getBuildingSegments().size() + " building segment; "); + public CubeSegment mergeSegments(CubeInstance cube, TSRange tsRange, SegmentRange segRange, boolean force) + throws IOException { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + return segAssist.mergeSegments(cube, tsRange, segRange, force); } } - private void checkCubeIsPartitioned(CubeInstance cube) { - if (cube.getDescriptor().getModel().getPartitionDesc().isPartitioned() == false) { - throw new IllegalStateException( - "there is no partition date column specified, only full build is supported"); + public void promoteNewlyBuiltSegments(CubeInstance cube, CubeSegment newSegment) throws IOException { + try (AutoLock lock = cubeMapLock.lockForWrite()) { + segAssist.promoteNewlyBuiltSegments(cube, newSegment); } } - /** - * After cube update, reload cube related cache - * - * @param cubeName - */ - public CubeInstance reloadCubeLocal(String cubeName) { - CubeInstance cubeInstance = reloadCubeLocalAt(CubeInstance.concatResourcePath(cubeName)); - if (cubeInstance != null) - Cuboid.clearCache(cubeInstance); - return cubeInstance; + public void validateNewSegments(CubeInstance cube, CubeSegment newSegments) { + segAssist.validateNewSegments(cube, newSegments); } - public void removeCubeLocal(String cubeName) { - CubeInstance cube = cubeMap.get(cubeName); - if (cube != null) { - cubeMap.removeLocal(cubeName); - for (CubeSegment segment : cube.getSegments()) { - usedStorageLocation.remove(segment.getUuid()); - } - Cuboid.clearCache(cube); - } + public List calculateHoles(String cubeName) { + return segAssist.calculateHoles(cubeName); } - public LookupStringTable getLookupTable(CubeSegment cubeSegment, JoinDesc join) { + private class SegmentAssist { + + CubeSegment appendSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange, + Map sourcePartitionOffsetStart, Map sourcePartitionOffsetEnd) + throws IOException { + checkInputRanges(tsRange, segRange); + checkBuildingSegment(cube); + + // fix start/end a bit + if (cube.getModel().getPartitionDesc().isPartitioned()) { + // if missing start, set it to where last time ends + CubeSegment last = cube.getLastSegment(); + if (last != null && !last.isOffsetCube() && tsRange.start.v == 0) { + tsRange = new TSRange(last.getTSRange().end.v, tsRange.end.v); + } + } else { + // full build + tsRange = null; + segRange = null; + } - String tableName = join.getPKSide().getTableIdentity(); - String[] pkCols = join.getPrimaryKey(); - String snapshotResPath = cubeSegment.getSnapshotResPath(tableName); - if (snapshotResPath == null) - throw new IllegalStateException("No snapshot for table '" + tableName + "' found on cube segment" - + cubeSegment.getCubeInstance().getName() + "/" + cubeSegment); + CubeSegment newSegment = newSegment(cube, tsRange, segRange); + newSegment.setSourcePartitionOffsetStart(sourcePartitionOffsetStart); + newSegment.setSourcePartitionOffsetEnd(sourcePartitionOffsetEnd); + validateNewSegments(cube, newSegment); - try { - SnapshotTable snapshot = getSnapshotManager().getSnapshotTable(snapshotResPath); - TableDesc tableDesc = getTableManager().getTableDesc(tableName, cubeSegment.getProject()); - return new LookupStringTable(tableDesc, pkCols, snapshot); - } catch (IOException e) { - throw new IllegalStateException( - "Failed to load lookup table " + tableName + " from snapshot " + snapshotResPath, e); + CubeUpdate cubeBuilder = new CubeUpdate(cube); + cubeBuilder.setToAddSegs(newSegment); + updateCube(cubeBuilder); + return newSegment; } - } - private CubeSegment newSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange) { - DataModelDesc modelDesc = cube.getModel(); + public CubeSegment refreshSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange) + throws IOException { + checkInputRanges(tsRange, segRange); + checkBuildingSegment(cube); - CubeSegment segment = new CubeSegment(); - segment.setUuid(UUID.randomUUID().toString()); - segment.setName(CubeSegment.makeSegmentName(tsRange, segRange, modelDesc)); - segment.setCreateTimeUTC(System.currentTimeMillis()); - segment.setCubeInstance(cube); + if (cube.getModel().getPartitionDesc().isPartitioned() == false) { + // full build + tsRange = null; + segRange = null; + } - // let full build range be backward compatible - if (tsRange == null && segRange == null) - tsRange = new TSRange(0L, Long.MAX_VALUE); + CubeSegment newSegment = newSegment(cube, tsRange, segRange); - segment.setTSRange(tsRange); - segment.setSegRange(segRange); - segment.setStatus(SegmentStatusEnum.NEW); - segment.setStorageLocationIdentifier(generateStorageLocation()); + Pair pair = cube.getSegments().fitInSegments(newSegment); + if (pair.getFirst() == false || pair.getSecond() == false) + throw new IllegalArgumentException("The new refreshing segment " + newSegment + + " does not match any existing segment in cube " + cube); - segment.setCubeInstance(cube); + if (segRange != null) { + CubeSegment toRefreshSeg = null; + for (CubeSegment cubeSegment : cube.getSegments()) { + if (cubeSegment.getSegRange().equals(segRange)) { + toRefreshSeg = cubeSegment; + break; + } + } - segment.validate(); - return segment; - } + if (toRefreshSeg == null) { + throw new IllegalArgumentException( + "For streaming cube, only one segment can be refreshed at one time"); + } - @VisibleForTesting - /*private*/ String generateStorageLocation() { - String namePrefix = config.getHBaseTableNamePrefix(); - String tableName = ""; - Random ran = new Random(); - do { - StringBuffer sb = new StringBuffer(); - sb.append(namePrefix); - for (int i = 0; i < HBASE_TABLE_LENGTH; i++) { - sb.append(ALPHA_NUM.charAt(ran.nextInt(ALPHA_NUM.length()))); + newSegment.setSourcePartitionOffsetStart(toRefreshSeg.getSourcePartitionOffsetStart()); + newSegment.setSourcePartitionOffsetEnd(toRefreshSeg.getSourcePartitionOffsetEnd()); } - tableName = sb.toString(); - } while (this.usedStorageLocation.containsValue(tableName)); - return tableName; - } - public void promoteNewlyBuiltSegments(CubeInstance cube, CubeSegment newSegment) throws IOException { - if (StringUtils.isBlank(newSegment.getStorageLocationIdentifier())) - throw new IllegalStateException( - "For cube " + cube + ", segment " + newSegment + " missing StorageLocationIdentifier"); + CubeUpdate cubeBuilder = new CubeUpdate(cube); + cubeBuilder.setToAddSegs(newSegment); + updateCube(cubeBuilder); - if (StringUtils.isBlank(newSegment.getLastBuildJobID())) - throw new IllegalStateException("For cube " + cube + ", segment " + newSegment + " missing LastBuildJobID"); - - if (isReady(newSegment) == true) { - logger.warn("For cube " + cube + ", segment " + newSegment + " state should be NEW but is READY"); + return newSegment; } - List tobe = cube.calculateToBeSegments(newSegment); + public CubeSegment mergeSegments(CubeInstance cube, TSRange tsRange, SegmentRange segRange, boolean force) + throws IOException { + if (cube.getSegments().isEmpty()) + throw new IllegalArgumentException("Cube " + cube + " has no segments"); + + checkInputRanges(tsRange, segRange); + checkBuildingSegment(cube); + checkCubeIsPartitioned(cube); + + if (cube.getSegments().getFirstSegment().isOffsetCube()) { + // offset cube, merge by date range? + if (segRange == null && tsRange != null) { + Pair pair = cube.getSegments(SegmentStatusEnum.READY) + .findMergeOffsetsByDateRange(tsRange, Long.MAX_VALUE); + if (pair == null) + throw new IllegalArgumentException( + "Find no segments to merge by " + tsRange + " for cube " + cube); + segRange = new SegmentRange(pair.getFirst().getSegRange().start, + pair.getSecond().getSegRange().end); + } + tsRange = null; + Preconditions.checkArgument(segRange != null); + } else { + segRange = null; + Preconditions.checkArgument(tsRange != null); + } + + CubeSegment newSegment = newSegment(cube, tsRange, segRange); + + Segments mergingSegments = cube.getMergingSegments(newSegment); + if (mergingSegments.size() <= 1) + throw new IllegalArgumentException("Range " + newSegment.getSegRange() + + " must contain at least 2 segments, but there is " + mergingSegments.size()); + + CubeSegment first = mergingSegments.get(0); + CubeSegment last = mergingSegments.get(mergingSegments.size() - 1); + if (force == false) { + for (int i = 0; i < mergingSegments.size() - 1; i++) { + if (!mergingSegments.get(i).getSegRange().connects(mergingSegments.get(i + 1).getSegRange())) + throw new IllegalStateException("Merging segments must not have gaps between " + + mergingSegments.get(i) + " and " + mergingSegments.get(i + 1)); + } + } + if (first.isOffsetCube()) { + newSegment.setSegRange(new SegmentRange(first.getSegRange().start, last.getSegRange().end)); + newSegment.setSourcePartitionOffsetStart(first.getSourcePartitionOffsetStart()); + newSegment.setSourcePartitionOffsetEnd(last.getSourcePartitionOffsetEnd()); + newSegment.setTSRange(null); + } else { + newSegment.setTSRange(new TSRange(mergingSegments.getTSStart(), mergingSegments.getTSEnd())); + newSegment.setSegRange(null); + } + + if (force == false) { + List emptySegment = Lists.newArrayList(); + for (CubeSegment seg : mergingSegments) { + if (seg.getSizeKB() == 0) { + emptySegment.add(seg.getName()); + } + } + + if (emptySegment.size() > 0) { + throw new IllegalArgumentException( + "Empty cube segment found, couldn't merge unless 'forceMergeEmptySegment' set to true: " + + emptySegment); + } + } - if (tobe.contains(newSegment) == false) - throw new IllegalStateException( - "For cube " + cube + ", segment " + newSegment + " is expected but not in the tobe " + tobe); + validateNewSegments(cube, newSegment); - newSegment.setStatus(SegmentStatusEnum.READY); + CubeUpdate cubeBuilder = new CubeUpdate(cube); + cubeBuilder.setToAddSegs(newSegment); + updateCube(cubeBuilder); - List toRemoveSegs = Lists.newArrayList(); - for (CubeSegment segment : cube.getSegments()) { - if (!tobe.contains(segment)) - toRemoveSegs.add(segment); + return newSegment; } - logger.info("Promoting cube " + cube + ", new segment " + newSegment + ", to remove segments " + toRemoveSegs); + private void checkInputRanges(TSRange tsRange, SegmentRange segRange) { + if (tsRange != null && segRange != null) { + throw new IllegalArgumentException( + "Build or refresh cube segment either by TSRange or by SegmentRange, not both."); + } + } - CubeUpdate cubeBuilder = new CubeUpdate(cube); - cubeBuilder.setToRemoveSegs(toRemoveSegs.toArray(new CubeSegment[toRemoveSegs.size()])) - .setToUpdateSegs(newSegment).setStatus(RealizationStatusEnum.READY); - updateCube(cubeBuilder); - } + private void checkBuildingSegment(CubeInstance cube) { + int maxBuldingSeg = cube.getConfig().getMaxBuildingSegments(); + if (cube.getBuildingSegments().size() >= maxBuldingSeg) { + throw new IllegalStateException( + "There is already " + cube.getBuildingSegments().size() + " building segment; "); + } + } - public void validateNewSegments(CubeInstance cube, CubeSegment newSegments) { - List tobe = cube.calculateToBeSegments(newSegments); - List newList = Arrays.asList(newSegments); - if (tobe.containsAll(newList) == false) { - throw new IllegalStateException("For cube " + cube + ", the new segments " + newList - + " do not fit in its current " + cube.getSegments() + "; the resulted tobe is " + tobe); + private void checkCubeIsPartitioned(CubeInstance cube) { + if (cube.getDescriptor().getModel().getPartitionDesc().isPartitioned() == false) { + throw new IllegalStateException( + "there is no partition date column specified, only full build is supported"); + } } - } - private boolean isReady(CubeSegment seg) { - return seg.getStatus() == SegmentStatusEnum.READY; - } + private CubeSegment newSegment(CubeInstance cube, TSRange tsRange, SegmentRange segRange) { + DataModelDesc modelDesc = cube.getModel(); - private void loadAllCubeInstance() throws IOException { - ResourceStore store = getStore(); - List paths = store.collectResourceRecursively(ResourceStore.CUBE_RESOURCE_ROOT, ".json"); + CubeSegment segment = new CubeSegment(); + segment.setUuid(UUID.randomUUID().toString()); + segment.setName(CubeSegment.makeSegmentName(tsRange, segRange, modelDesc)); + segment.setCreateTimeUTC(System.currentTimeMillis()); + segment.setCubeInstance(cube); - logger.info("Loading Cube from folder " + store.getReadableResourcePath(ResourceStore.CUBE_RESOURCE_ROOT)); + // let full build range be backward compatible + if (tsRange == null && segRange == null) + tsRange = new TSRange(0L, Long.MAX_VALUE); - int succeed = 0; - int fail = 0; - for (String path : paths) { - CubeInstance cube = reloadCubeLocalAt(path); - if (cube == null) { - fail++; - } else { - succeed++; - } + segment.setTSRange(tsRange); + segment.setSegRange(segRange); + segment.setStatus(SegmentStatusEnum.NEW); + segment.setStorageLocationIdentifier(generateStorageLocation()); + + segment.setCubeInstance(cube); + + segment.validate(); + return segment; } - logger.info("Loaded " + succeed + " cubes, fail on " + fail + " cubes"); - } + public void promoteNewlyBuiltSegments(CubeInstance cube, CubeSegment newSegment) throws IOException { + if (StringUtils.isBlank(newSegment.getStorageLocationIdentifier())) + throw new IllegalStateException( + "For cube " + cube + ", segment " + newSegment + " missing StorageLocationIdentifier"); - private CubeInstance reloadCubeLocalAt(String path) { - ResourceStore store = getStore(); - CubeInstance cube; + if (StringUtils.isBlank(newSegment.getLastBuildJobID())) + throw new IllegalStateException( + "For cube " + cube + ", segment " + newSegment + " missing LastBuildJobID"); - try { - cube = store.getResource(path, CubeInstance.class, CUBE_SERIALIZER); - if (cube == null) { - return cube; + if (isReady(newSegment) == true) { + logger.warn("For cube " + cube + ", segment " + newSegment + " state should be NEW but is READY"); } - String cubeName = cube.getName(); - checkState(StringUtils.isNotBlank(cubeName), "cube (at %s) name must not be blank", path); + List tobe = cube.calculateToBeSegments(newSegment); - CubeDesc cubeDesc = CubeDescManager.getInstance(config).getCubeDesc(cube.getDescName()); - checkNotNull(cubeDesc, "cube descriptor '%s' (for cube '%s') not found", cube.getDescName(), cubeName); + if (tobe.contains(newSegment) == false) + throw new IllegalStateException( + "For cube " + cube + ", segment " + newSegment + " is expected but not in the tobe " + tobe); - if (!cubeDesc.getError().isEmpty()) { - cube.setStatus(RealizationStatusEnum.DESCBROKEN); - logger.error("cube descriptor {} (for cube '{}') is broken", cubeDesc.getResourcePath(), cubeName); - for (String error : cubeDesc.getError()) { - logger.error("Error: {}", error); - } - } else if (cube.getStatus() == RealizationStatusEnum.DESCBROKEN) { - cube.setStatus(RealizationStatusEnum.DISABLED); - logger.info("cube {} changed from DESCBROKEN to DISABLED", cubeName); + newSegment.setStatus(SegmentStatusEnum.READY); + + List toRemoveSegs = Lists.newArrayList(); + for (CubeSegment segment : cube.getSegments()) { + if (!tobe.contains(segment)) + toRemoveSegs.add(segment); } - cube.setConfig((KylinConfigExt) cubeDesc.getConfig()); - cubeMap.putLocal(cubeName, cube); + logger.info( + "Promoting cube " + cube + ", new segment " + newSegment + ", to remove segments " + toRemoveSegs); - for (CubeSegment segment : cube.getSegments()) { - usedStorageLocation.put(segment.getUuid(), segment.getStorageLocationIdentifier()); + CubeUpdate cubeBuilder = new CubeUpdate(cube); + cubeBuilder.setToRemoveSegs(toRemoveSegs.toArray(new CubeSegment[toRemoveSegs.size()])) + .setToUpdateSegs(newSegment).setStatus(RealizationStatusEnum.READY); + updateCube(cubeBuilder); + } + + public void validateNewSegments(CubeInstance cube, CubeSegment newSegments) { + List tobe = cube.calculateToBeSegments(newSegments); + List newList = Arrays.asList(newSegments); + if (tobe.containsAll(newList) == false) { + throw new IllegalStateException("For cube " + cube + ", the new segments " + newList + + " do not fit in its current " + cube.getSegments() + "; the resulted tobe is " + tobe); } + } - logger.info("Reloaded cube {} being {} having {} segments", cubeName, cube, cube.getSegments().size()); - return cube; + /** + * Calculate the holes (gaps) in segments. + * @param cubeName + * @return + */ + public List calculateHoles(String cubeName) { + List holes = Lists.newArrayList(); + final CubeInstance cube = getCube(cubeName); + DataModelDesc modelDesc = cube.getModel(); + Preconditions.checkNotNull(cube); + final List segments = cube.getSegments(); + logger.info("totally " + segments.size() + " cubeSegments"); + if (segments.size() == 0) { + return holes; + } - } catch (Exception e) { - logger.error("Error during load cube instance, skipping : " + path, e); - return null; + Collections.sort(segments); + for (int i = 0; i < segments.size() - 1; ++i) { + CubeSegment first = segments.get(i); + CubeSegment second = segments.get(i + 1); + if (first.getSegRange().connects(second.getSegRange())) + continue; + + if (first.getSegRange().apartBefore(second.getSegRange())) { + CubeSegment hole = new CubeSegment(); + hole.setCubeInstance(cube); + if (first.isOffsetCube()) { + hole.setSegRange(new SegmentRange(first.getSegRange().end, second.getSegRange().start)); + hole.setSourcePartitionOffsetStart(first.getSourcePartitionOffsetEnd()); + hole.setSourcePartitionOffsetEnd(second.getSourcePartitionOffsetStart()); + hole.setName(CubeSegment.makeSegmentName(null, hole.getSegRange(), modelDesc)); + } else { + hole.setTSRange(new TSRange(first.getTSRange().end.v, second.getTSRange().start.v)); + hole.setName(CubeSegment.makeSegmentName(hole.getTSRange(), null, modelDesc)); + } + holes.add(hole); + } + } + return holes; } + } - private TableMetadataManager getTableManager() { - return TableMetadataManager.getInstance(config); + // ============================================================================ + // Dictionary/Snapshot related methods + // ============================================================================ + + public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable) + throws IOException { + return dictAssist.buildDictionary(cubeSeg, col, inpTable); } - private DictionaryManager getDictionaryManager() { - return DictionaryManager.getInstance(config); + public DictionaryInfo saveDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable, + Dictionary dict) throws IOException { + return dictAssist.saveDictionary(cubeSeg, col, inpTable, dict); } - private SnapshotManager getSnapshotManager() { - return SnapshotManager.getInstance(config); + /** + * return null if no dictionary for given column + */ + public Dictionary getDictionary(CubeSegment cubeSeg, TblColRef col) { + return dictAssist.getDictionary(cubeSeg, col); } - private ResourceStore getStore() { - return ResourceStore.getStore(this.config); + public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable) throws IOException { + return dictAssist.buildSnapshotTable(cubeSeg, lookupTable); } - @Override - public RealizationType getRealizationType() { - return RealizationType.CUBE; + public LookupStringTable getLookupTable(CubeSegment cubeSegment, JoinDesc join) { + return dictAssist.getLookupTable(cubeSegment, join); } - @Override - public IRealization getRealization(String name) { - return getCube(name); + //UHC (ultra high cardinality column): contain the ShardByColumns and the GlobalDictionaryColumns + public int[] getUHCIndex(CubeDesc cubeDesc) throws IOException { + return dictAssist.getUHCIndex(cubeDesc); } - /** - * Calculate the holes (gaps) in segments. - * @param cubeName - * @return - */ - public List calculateHoles(String cubeName) { - List holes = Lists.newArrayList(); - final CubeInstance cube = getCube(cubeName); - DataModelDesc modelDesc = cube.getModel(); - Preconditions.checkNotNull(cube); - final List segments = cube.getSegments(); - logger.info("totally " + segments.size() + " cubeSegments"); - if (segments.size() == 0) { - return holes; + private class DictionaryAssist { + public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable) + throws IOException { + CubeDesc cubeDesc = cubeSeg.getCubeDesc(); + if (!cubeDesc.getAllColumnsNeedDictionaryBuilt().contains(col)) + return null; + + String builderClass = cubeDesc.getDictionaryBuilderClass(col); + DictionaryInfo dictInfo = getDictionaryManager().buildDictionary(col, inpTable, builderClass); + + saveDictionaryInfo(cubeSeg, col, dictInfo); + return dictInfo; } - Collections.sort(segments); - for (int i = 0; i < segments.size() - 1; ++i) { - CubeSegment first = segments.get(i); - CubeSegment second = segments.get(i + 1); - if (first.getSegRange().connects(second.getSegRange())) - continue; - - if (first.getSegRange().apartBefore(second.getSegRange())) { - CubeSegment hole = new CubeSegment(); - hole.setCubeInstance(cube); - if (first.isOffsetCube()) { - hole.setSegRange(new SegmentRange(first.getSegRange().end, second.getSegRange().start)); - hole.setSourcePartitionOffsetStart(first.getSourcePartitionOffsetEnd()); - hole.setSourcePartitionOffsetEnd(second.getSourcePartitionOffsetStart()); - hole.setName(CubeSegment.makeSegmentName(null, hole.getSegRange(), modelDesc)); - } else { - hole.setTSRange(new TSRange(first.getTSRange().end.v, second.getTSRange().start.v)); - hole.setName(CubeSegment.makeSegmentName(hole.getTSRange(), null, modelDesc)); - } - holes.add(hole); + public DictionaryInfo saveDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable, + Dictionary dict) throws IOException { + CubeDesc cubeDesc = cubeSeg.getCubeDesc(); + if (!cubeDesc.getAllColumnsNeedDictionaryBuilt().contains(col)) + return null; + + DictionaryInfo dictInfo = getDictionaryManager().saveDictionary(col, inpTable, dict); + + saveDictionaryInfo(cubeSeg, col, dictInfo); + return dictInfo; + } + + private void saveDictionaryInfo(CubeSegment cubeSeg, TblColRef col, DictionaryInfo dictInfo) + throws IOException { + if (dictInfo != null) { + Dictionary dict = dictInfo.getDictionaryObject(); + cubeSeg.putDictResPath(col, dictInfo.getResourcePath()); + cubeSeg.getRowkeyStats().add(new Object[] { col.getIdentity(), dict.getSize(), dict.getSizeOfId() }); + + CubeUpdate update = new CubeUpdate(cubeSeg.getCubeInstance()); + update.setToUpdateSegs(cubeSeg); + updateCube(update); } } - return holes; - } - private final String GLOBAL_DICTIONNARY_CLASS = "org.apache.kylin.dict.GlobalDictionaryBuilder"; + /** + * return null if no dictionary for given column + */ + @SuppressWarnings("unchecked") + public Dictionary getDictionary(CubeSegment cubeSeg, TblColRef col) { + DictionaryInfo info = null; + try { + DictionaryManager dictMgr = getDictionaryManager(); + String dictResPath = cubeSeg.getDictResPath(col); + if (dictResPath == null) + return null; + + info = dictMgr.getDictionaryInfo(dictResPath); + if (info == null) + throw new IllegalStateException("No dictionary found by " + dictResPath + + ", invalid cube state; cube segment" + cubeSeg + ", col " + col); + } catch (IOException e) { + throw new IllegalStateException("Failed to get dictionary for cube segment" + cubeSeg + ", col" + col, + e); + } + return (Dictionary) info.getDictionaryObject(); + } - //UHC (ultra high cardinality column): contain the ShardByColumns and the GlobalDictionaryColumns - public int[] getUHCIndex(CubeDesc cubeDesc) throws IOException { - List dictCols = Lists.newArrayList(cubeDesc.getAllColumnsNeedDictionaryBuilt()); - int[] uhcIndex = new int[dictCols.size()]; - - //add GlobalDictionaryColumns - List dictionaryDescList = cubeDesc.getDictionaries(); - if (dictionaryDescList != null) { - for (DictionaryDesc dictionaryDesc : dictionaryDescList) { - if (dictionaryDesc.getBuilderClass() != null - && dictionaryDesc.getBuilderClass().equalsIgnoreCase(GLOBAL_DICTIONNARY_CLASS)) { - for (int i = 0; i < dictCols.size(); i++) { - if (dictCols.get(i).equals(dictionaryDesc.getColumnRef())) { - uhcIndex[i] = 1; - break; + public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable) throws IOException { + TableMetadataManager metaMgr = getTableManager(); + SnapshotManager snapshotMgr = getSnapshotManager(); + + TableDesc tableDesc = new TableDesc(metaMgr.getTableDesc(lookupTable, cubeSeg.getProject())); + IReadableTable hiveTable = SourceFactory.createReadableTable(tableDesc); + SnapshotTable snapshot = snapshotMgr.buildSnapshot(hiveTable, tableDesc); + + cubeSeg.putSnapshotResPath(lookupTable, snapshot.getResourcePath()); + CubeUpdate cubeBuilder = new CubeUpdate(cubeSeg.getCubeInstance()); + cubeBuilder.setToUpdateSegs(cubeSeg); + updateCube(cubeBuilder); + + return snapshot; + } + + public LookupStringTable getLookupTable(CubeSegment cubeSegment, JoinDesc join) { + + String tableName = join.getPKSide().getTableIdentity(); + String[] pkCols = join.getPrimaryKey(); + String snapshotResPath = cubeSegment.getSnapshotResPath(tableName); + if (snapshotResPath == null) + throw new IllegalStateException("No snapshot for table '" + tableName + "' found on cube segment" + + cubeSegment.getCubeInstance().getName() + "/" + cubeSegment); + + try { + SnapshotTable snapshot = getSnapshotManager().getSnapshotTable(snapshotResPath); + TableDesc tableDesc = getTableManager().getTableDesc(tableName, cubeSegment.getProject()); + return new LookupStringTable(tableDesc, pkCols, snapshot); + } catch (IOException e) { + throw new IllegalStateException( + "Failed to load lookup table " + tableName + " from snapshot " + snapshotResPath, e); + } + } + + private final String GLOBAL_DICTIONNARY_CLASS = "org.apache.kylin.dict.GlobalDictionaryBuilder"; + + //UHC (ultra high cardinality column): contain the ShardByColumns and the GlobalDictionaryColumns + public int[] getUHCIndex(CubeDesc cubeDesc) throws IOException { + List dictCols = Lists.newArrayList(cubeDesc.getAllColumnsNeedDictionaryBuilt()); + int[] uhcIndex = new int[dictCols.size()]; + + //add GlobalDictionaryColumns + List dictionaryDescList = cubeDesc.getDictionaries(); + if (dictionaryDescList != null) { + for (DictionaryDesc dictionaryDesc : dictionaryDescList) { + if (dictionaryDesc.getBuilderClass() != null + && dictionaryDesc.getBuilderClass().equalsIgnoreCase(GLOBAL_DICTIONNARY_CLASS)) { + for (int i = 0; i < dictCols.size(); i++) { + if (dictCols.get(i).equals(dictionaryDesc.getColumnRef())) { + uhcIndex[i] = 1; + break; + } } } } } - } - //add ShardByColumns - Set shardByColumns = cubeDesc.getShardByColumns(); - for (int i = 0; i < dictCols.size(); i++) { - if (shardByColumns.contains(dictCols.get(i))) { - uhcIndex[i] = 1; + //add ShardByColumns + Set shardByColumns = cubeDesc.getShardByColumns(); + for (int i = 0; i < dictCols.size(); i++) { + if (shardByColumns.contains(dictCols.get(i))) { + uhcIndex[i] = 1; + } } - } - return uhcIndex; + return uhcIndex; + } } + } diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cli/CubeSignatureRefresher.java b/core-cube/src/main/java/org/apache/kylin/cube/cli/CubeSignatureRefresher.java index d07c93bfced..2eaebb1a9a5 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/cli/CubeSignatureRefresher.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/cli/CubeSignatureRefresher.java @@ -97,7 +97,7 @@ private void updateCubeDesc(CubeDesc cubeDesc) { String calculatedSign = cubeDesc.calculateSignature(); if (cubeDesc.getSignature() == null || (!cubeDesc.getSignature().equals(calculatedSign))) { cubeDesc.setSignature(calculatedSign); - store.putResource(cubeDesc.getResourcePath(), cubeDesc, CubeDescManager.CUBE_DESC_SERIALIZER); + store.putResource(cubeDesc.getResourcePath(), cubeDesc, CubeDesc.newSerializerForLowLevelAccess()); updatedResources.add(cubeDesc.getResourcePath()); } } catch (Exception e) { diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java index f39432b4c32..efb9c59b15a 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java @@ -47,6 +47,7 @@ import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.KylinConfigExt; import org.apache.kylin.common.KylinVersion; +import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.RootPersistentEntity; import org.apache.kylin.common.util.Array; @@ -58,13 +59,13 @@ import org.apache.kylin.metadata.MetadataConstants; import org.apache.kylin.metadata.model.ColumnDesc; import org.apache.kylin.metadata.model.DataModelDesc; +import org.apache.kylin.metadata.model.DataModelManager; import org.apache.kylin.metadata.model.FunctionDesc; import org.apache.kylin.metadata.model.IEngineAware; import org.apache.kylin.metadata.model.IStorageAware; import org.apache.kylin.metadata.model.JoinDesc; import org.apache.kylin.metadata.model.JoinTableDesc; import org.apache.kylin.metadata.model.MeasureDesc; -import org.apache.kylin.metadata.model.DataModelManager; import org.apache.kylin.metadata.model.TableRef; import org.apache.kylin.metadata.model.TblColRef; import org.apache.kylin.metadata.project.ProjectInstance; @@ -90,6 +91,11 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware { private static final Logger logger = LoggerFactory.getLogger(CubeDesc.class); + // Use with care! Normally you should go to CubeDescManager and don't need this. + public static JsonSerializer newSerializerForLowLevelAccess() { + return new JsonSerializer<>(CubeDesc.class); + } + public static class CannotFilterExtendedColumnException extends RuntimeException { public CannotFilterExtendedColumnException(TblColRef tblColRef) { super(tblColRef == null ? "null" : tblColRef.getCanonicalName()); @@ -122,6 +128,8 @@ public String toString() { } } + + // ============================================================================ private KylinConfigExt config; private DataModelDesc model; @@ -183,6 +191,9 @@ public String toString() { @JsonInclude(JsonInclude.Include.NON_NULL) private int parentForward = 3; + // Error messages during resolving json metadata + private List errors = new ArrayList(); + private LinkedHashSet allColumns = new LinkedHashSet<>(); private LinkedHashSet allColumnDescs = new LinkedHashSet<>(); private LinkedHashSet dimensionColumns = new LinkedHashSet<>(); @@ -194,6 +205,11 @@ public String toString() { transient volatile private CuboidScheduler cuboidScheduler = null; + @Override + public String resourceName() { + return name; + } + public boolean isEnableSharding() { //in the future may extend to other storage that is shard-able return storageType != IStorageAware.ID_HBASE && storageType != IStorageAware.ID_HYBRID; @@ -203,11 +219,6 @@ public Set getShardByColumns() { return getRowkey().getShardByColumns(); } - /** - * Error messages during resolving json metadata - */ - private List errors = new ArrayList(); - /** * @return all columns this cube can support, including derived */ @@ -311,7 +322,7 @@ public Map, List> getHostToDerivedInfo(List TABLE_SERIALIZER = new JsonSerializer(TableDesc.class); - public static final Serializer TABLE_EXT_SERIALIZER = new JsonSerializer( + private static final Serializer TABLE_EXT_SERIALIZER = new JsonSerializer( TableExtDesc.class); - public static final Serializer EXTERNAL_FILTER_DESC_SERIALIZER = new JsonSerializer( - ExternalFilterDesc.class); public static TableMetadataManager getInstance(KylinConfig config) { return config.getManager(TableMetadataManager.class); @@ -72,15 +73,28 @@ static TableMetadataManager newInstance(KylinConfig config) throws IOException { // ============================================================================ private KylinConfig config; + // table name ==> SourceTable private CaseInsensitiveStringCache srcTableMap; + private CachedCrudAssist srcTableCrud; + private AutoReadWriteLock srcTableMapLock = new AutoReadWriteLock(); + // name => SourceTableExt - private CaseInsensitiveStringCache srcTableExtMap; - // name => External Filter Desc + private CaseInsensitiveStringCache srcExtMap; + private CachedCrudAssist srcExtCrud; + private AutoReadWriteLock srcExtMapLock = new AutoReadWriteLock(); + + // name => ExternalFilterDesc private CaseInsensitiveStringCache extFilterMap; + private CachedCrudAssist extFilterCrud; + private AutoReadWriteLock extFilterMapLock = new AutoReadWriteLock(); + + private TableMetadataManager(KylinConfig cfg) throws IOException { + this.config = cfg; - private TableMetadataManager(KylinConfig config) throws IOException { - init(config); + initSrcTable(); + initSrcExt(); + initExtFilter(); } public KylinConfig getConfig() { @@ -91,42 +105,114 @@ public ResourceStore getStore() { return ResourceStore.getStore(this.config); } - public List listAllTables(String prj) { - return Lists.newArrayList(getAllTablesMap(prj).values()); + // ============================================================================ + // TableDesc methods + // ============================================================================ + + private void initSrcTable() throws IOException { + this.srcTableMap = new CaseInsensitiveStringCache<>(config, "table"); + this.srcTableCrud = new CachedCrudAssist(getStore(), ResourceStore.TABLE_RESOURCE_ROOT, + TableDesc.class, srcTableMap) { + @Override + protected TableDesc initEntityAfterReload(TableDesc t, String resourceName) { + String prj = TableDesc.parseResourcePath(resourceName).getSecond(); + t.init(prj); + return t; + } + }; + srcTableCrud.reloadAll(); + Broadcaster.getInstance(config).registerListener(new SrcTableSyncListener(), "table"); } - public List listAllExternalFilters() { - return Lists.newArrayList(extFilterMap.values()); + private class SrcTableSyncListener extends Broadcaster.Listener { + @Override + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + if (event == Event.DROP) + srcTableMap.removeLocal(cacheKey); + else + srcTableCrud.reloadQuietly(cacheKey); + } + + Pair pair = TableDesc.parseResourcePath(cacheKey); + String table = pair.getFirst(); + String prj = pair.getSecond(); + + if (prj == null) { + for (ProjectInstance p : ProjectManager.getInstance(config).findProjectsByTable(table)) { + broadcaster.notifyProjectSchemaUpdate(p.getName()); + } + } else { + broadcaster.notifyProjectSchemaUpdate(prj); + } + } + } + + public List listAllTables(String prj) { + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + return Lists.newArrayList(getAllTablesMap(prj).values()); + } } public Map getAllTablesMap(String prj) { - //TODO prj == null case is now only used by test case and CubeMetaIngester - //should refactor these test case and tool ASAP and stop supporting null case - if (prj == null) { - Map globalTables = new LinkedHashMap<>(); + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + //TODO prj == null case is now only used by test case and CubeMetaIngester + //should refactor these test case and tool ASAP and stop supporting null case + if (prj == null) { + Map globalTables = new LinkedHashMap<>(); - for (TableDesc t : srcTableMap.values()) { - globalTables.put(t.getIdentity(), t); + for (TableDesc t : srcTableMap.values()) { + globalTables.put(t.getIdentity(), t); + } + return globalTables; } - return globalTables; - } - - ProjectInstance project = ProjectManager.getInstance(config).getProject(prj); - Set prjTableNames = project.getTables(); - - Map ret = new LinkedHashMap<>(); - for (String tableName : prjTableNames) { - String tableIdentity = getTableIdentity(tableName); - ret.put(tableIdentity, getProjectSpecificTableDesc(tableIdentity, prj)); + + ProjectInstance project = ProjectManager.getInstance(config).getProject(prj); + Set prjTableNames = project.getTables(); + + Map ret = new LinkedHashMap<>(); + for (String tableName : prjTableNames) { + String tableIdentity = getTableIdentity(tableName); + ret.put(tableIdentity, getProjectSpecificTableDesc(tableIdentity, prj)); + } + return ret; } - return ret; } /** * Get TableDesc by name */ public TableDesc getTableDesc(String tableName, String prj) { - return getProjectSpecificTableDesc(getTableIdentity(tableName), prj); + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + return getProjectSpecificTableDesc(getTableIdentity(tableName), prj); + } + } + + /** + * Make sure the returned table desc is project-specific. + * + * All locks on srcTableMapLock are WRITE LOCKS because of this method!! + */ + private TableDesc getProjectSpecificTableDesc(String fullTableName, String prj) { + String key = mapKey(fullTableName, prj); + TableDesc result = srcTableMap.get(key); + + if (result == null) { + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + result = srcTableMap.get(mapKey(fullTableName, null)); + if (result != null) { + result = new TableDesc(result);// deep copy of global tabledesc + + result.setLastModified(0); + result.setProject(prj); + result.setBorrowedFromGlobal(true); + + srcTableMap.putLocal(key, result); + } + } + } + return result; } /** @@ -139,55 +225,87 @@ private String getTableIdentity(String tableName) { return tableName.toUpperCase(); } + public void saveSourceTable(TableDesc srcTable, String prj) throws IOException { + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + srcTable.init(prj); + srcTableCrud.save(srcTable); + } + } + + public void removeSourceTable(String tableIdentity, String prj) throws IOException { + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + TableDesc t = getTableDesc(tableIdentity, prj); + if (t == null) + return; + + srcTableCrud.delete(t); + } + } + /** * the project-specific table desc will be expand by computed columns from the projects' models * when the projects' model list changed, project-specific table should be reset and get expanded * again */ public void resetProjectSpecificTableDesc(String prj) throws IOException { - ProjectInstance project = ProjectManager.getInstance(config).getProject(prj); - for (String tableName : project.getTables()) { - String tableIdentity = getTableIdentity(tableName); - String key = mapKey(tableIdentity, prj); - TableDesc originTableDesc = srcTableMap.get(key); - if (originTableDesc == null) { - continue; - } + try (AutoLock lock = srcTableMapLock.lockForWrite()) { + ProjectInstance project = ProjectManager.getInstance(config).getProject(prj); + for (String tableName : project.getTables()) { + String tableIdentity = getTableIdentity(tableName); + String key = mapKey(tableIdentity, prj); + TableDesc originTableDesc = srcTableMap.get(key); + if (originTableDesc == null) { + continue; + } - if (originTableDesc.isBorrowedFromGlobal()) { - srcTableMap.removeLocal(key);//delete it so that getProjectSpecificTableDesc will create again - } else { - String s = originTableDesc.getResourcePath(); - TableDesc tableDesc = reloadSourceTableAt(s); - srcTableMap.putLocal(key, tableDesc); + if (originTableDesc.isBorrowedFromGlobal()) { + srcTableMap.removeLocal(key);//delete it so that getProjectSpecificTableDesc will create again + } else { + srcTableCrud.reload(key); + } } } } - /** - * make sure the returned table desc is project-specific - */ - private TableDesc getProjectSpecificTableDesc(String fullTableName, String prj) { - String key = mapKey(fullTableName, prj); - TableDesc result = srcTableMap.get(key); + private String mapKey(String identity, String prj) { + return TableDesc.makeResourceName(identity, prj); + } - if (result == null) { - result = srcTableMap.get(mapKey(fullTableName, null)); - if (result != null) { - result = new TableDesc(result);// deep copy of global tabledesc + // ============================================================================ + // TableExtDesc methods + // ============================================================================ - result.setProject(prj); - result.setBorrowedFromGlobal(true); + private void initSrcExt() throws IOException { + this.srcExtMap = new CaseInsensitiveStringCache<>(config, "table_ext"); + this.srcExtCrud = new CachedCrudAssist(getStore(), ResourceStore.TABLE_EXD_RESOURCE_ROOT, + TableExtDesc.class, srcExtMap) { + @Override + protected TableExtDesc initEntityAfterReload(TableExtDesc t, String resourceName) { + // convert old tableExt json to new one + if (t.getIdentity() == null) { + t = convertOldTableExtToNewer(resourceName); + } - srcTableMap.putLocal(key, result); + String prj = TableDesc.parseResourcePath(resourceName).getSecond(); + t.init(prj); + return t; } - } - return result; + }; + srcExtCrud.reloadAll(); + Broadcaster.getInstance(config).registerListener(new SrcTableExtSyncListener(), "table_ext"); } - public ExternalFilterDesc getExtFilterDesc(String filterTableName) { - ExternalFilterDesc result = extFilterMap.get(filterTableName); - return result; + private class SrcTableExtSyncListener extends Broadcaster.Listener { + @Override + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { + try (AutoLock lock = srcExtMapLock.lockForWrite()) { + if (event == Event.DROP) + srcExtMap.removeLocal(cacheKey); + else + srcExtCrud.reloadQuietly(cacheKey); + } + } } /** @@ -205,233 +323,89 @@ public TableExtDesc getTableExt(String tableName, String prj) { } public TableExtDesc getTableExt(TableDesc t) { - TableExtDesc result = srcTableExtMap.get(mapKey(t.getIdentity(), t.getProject())); + try (AutoLock lock = srcExtMapLock.lockForRead()) { + TableExtDesc result = srcExtMap.get(mapKey(t.getIdentity(), t.getProject())); - if (null == result) { - //TODO: notice the table ext is not project-specific, seems not necessary at all - result = srcTableExtMap.get(mapKey(t.getIdentity(), null)); - } + if (null == result) { + //TODO: notice the table ext is not project-specific, seems not necessary at all + result = srcExtMap.get(mapKey(t.getIdentity(), null)); + } - // avoid returning null, since the TableDesc exists - if (null == result) { - result = new TableExtDesc(); - result.setIdentity(t.getIdentity()); - result.setUuid(UUID.randomUUID().toString()); - result.setLastModified(0); - result.init(t.getProject()); - srcTableExtMap.put(mapKey(t.getIdentity(), t.getProject()), result); + // avoid returning null, since the TableDesc exists + if (null == result) { + result = new TableExtDesc(); + result.setIdentity(t.getIdentity()); + result.setUuid(UUID.randomUUID().toString()); + result.setLastModified(0); + result.init(t.getProject()); + srcExtMap.put(mapKey(t.getIdentity(), t.getProject()), result); + } + return result; } - return result; } public void saveTableExt(TableExtDesc tableExt, String prj) throws IOException { - if (tableExt.getUuid() == null || tableExt.getIdentity() == null) { - throw new IllegalArgumentException(); - } - - // updating a legacy global table - if (tableExt.getProject() == null) { - if (getTableExt(tableExt.getIdentity(), prj).getProject() != null) - throw new IllegalStateException( - "Updating a legacy global TableExtDesc while a project level version exists: " - + tableExt.getIdentity() + ", " + prj); - prj = tableExt.getProject(); - } - - tableExt.init(prj); - - String path = TableExtDesc.concatResourcePath(tableExt.getIdentity(), prj); - - ResourceStore store = getStore(); - - TableExtDesc t = store.getResource(path, TableExtDesc.class, TABLE_EXT_SERIALIZER); - if (t != null && t.getIdentity() == null) - store.deleteResource(path); - - store.putResource(path, tableExt, TABLE_EXT_SERIALIZER); - srcTableExtMap.put(mapKey(tableExt.getIdentity(), tableExt.getProject()), tableExt); - } - - public void removeTableExt(String tableName, String prj) throws IOException { - // note, here assume always delete TableExtDesc first, then TableDesc - TableExtDesc t = getTableExt(tableName, prj); - if (t == null) - return; - - String path = TableExtDesc.concatResourcePath(t.getIdentity(), t.getProject()); - getStore().deleteResource(path); - srcTableExtMap.remove(mapKey(t.getIdentity(), t.getProject())); - } - - public void saveSourceTable(TableDesc srcTable, String prj) throws IOException { - if (srcTable.getUuid() == null || srcTable.getIdentity() == null) { - throw new IllegalArgumentException(); - } - - srcTable.init(prj); - - String path = srcTable.getResourcePath(); - getStore().putResource(path, srcTable, TABLE_SERIALIZER); - - srcTableMap.put(mapKey(srcTable.getIdentity(), prj), srcTable); - } - - public void removeSourceTable(String tableIdentity, String prj) throws IOException { - TableDesc t = getTableDesc(tableIdentity, prj); - if (t == null) - return; - - String path = t.getResourcePath(); - getStore().deleteResource(path); - srcTableMap.remove(mapKey(t.getIdentity(), t.getProject())); - } - - public void saveExternalFilter(ExternalFilterDesc desc) throws IOException { - if (desc.getUuid() == null) { - throw new IllegalArgumentException("UUID not set."); - } - String path = desc.getResourcePath(); - getStore().putResource(path, desc, EXTERNAL_FILTER_DESC_SERIALIZER); - desc = reloadExternalFilterAt(path); - extFilterMap.put(desc.getName(), desc); - - } - - public void removeExternalFilter(String name) throws IOException { - String path = ExternalFilterDesc.concatResourcePath(name); - getStore().deleteResource(path); - extFilterMap.remove(name); - - } - - private void init(KylinConfig config) throws IOException { - this.config = config; - this.srcTableMap = new CaseInsensitiveStringCache<>(config, "table"); - this.srcTableExtMap = new CaseInsensitiveStringCache<>(config, "table_ext"); - this.extFilterMap = new CaseInsensitiveStringCache<>(config, "external_filter"); - - reloadAllSourceTable(); - reloadAllTableExt(); - reloadAllExternalFilter(); - - // touch lower level metadata before registering my listener - Broadcaster.getInstance(config).registerListener(new SrcTableSyncListener(), "table"); - Broadcaster.getInstance(config).registerListener(new SrcTableExtSyncListener(), "table_ext"); - Broadcaster.getInstance(config).registerListener(new ExtFilterSyncListener(), "external_filter"); - } - - private class SrcTableSyncListener extends Broadcaster.Listener { - - @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) - throws IOException { - if (event == Event.DROP) - srcTableMap.removeLocal(cacheKey); - else - reloadSourceTableAt(TableDesc.concatRawResourcePath(cacheKey)); - - Pair pair = TableDesc.parseResourcePath(cacheKey); - String table = pair.getFirst(); - String prj = pair.getSecond(); - - if (prj == null) { - for (ProjectInstance p : ProjectManager.getInstance(config).findProjectsByTable(table)) { - broadcaster.notifyProjectSchemaUpdate(p.getName()); - } - } else { - broadcaster.notifyProjectSchemaUpdate(prj); + try (AutoLock lock = srcExtMapLock.lockForWrite()) { + if (tableExt.getUuid() == null || tableExt.getIdentity() == null) { + throw new IllegalArgumentException(); } - } - } - private class SrcTableExtSyncListener extends Broadcaster.Listener { + // updating a legacy global table + if (tableExt.getProject() == null) { + if (getTableExt(tableExt.getIdentity(), prj).getProject() != null) + throw new IllegalStateException( + "Updating a legacy global TableExtDesc while a project level version exists: " + + tableExt.getIdentity() + ", " + prj); + prj = tableExt.getProject(); + } - @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) - throws IOException { - if (event == Event.DROP) - srcTableExtMap.removeLocal(cacheKey); - else - reloadTableExtAt(TableExtDesc.concatRawResourcePath(cacheKey)); - } - } + tableExt.init(prj); - private class ExtFilterSyncListener extends Broadcaster.Listener { + // what is this doing?? + String path = TableExtDesc.concatResourcePath(tableExt.getIdentity(), prj); + ResourceStore store = getStore(); + TableExtDesc t = store.getResource(path, TableExtDesc.class, TABLE_EXT_SERIALIZER); + if (t != null && t.getIdentity() == null) + store.deleteResource(path); - @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) - throws IOException { - if (event == Event.DROP) - extFilterMap.removeLocal(cacheKey); - else - reloadExtFilter(cacheKey); + srcExtCrud.save(tableExt); } } - private void reloadAllTableExt() throws IOException { - ResourceStore store = getStore(); - logger.debug("Reloading Table_exd info from folder " - + store.getReadableResourcePath(ResourceStore.TABLE_EXD_RESOURCE_ROOT)); - - srcTableExtMap.clear(); + public void removeTableExt(String tableName, String prj) throws IOException { + try (AutoLock lock = srcExtMapLock.lockForWrite()) { + // note, here assume always delete TableExtDesc first, then TableDesc + TableExtDesc t = getTableExt(tableName, prj); + if (t == null) + return; - List paths = store.collectResourceRecursively(ResourceStore.TABLE_EXD_RESOURCE_ROOT, - MetadataConstants.FILE_SURFIX); - for (String path : paths) { - reloadTableExtAt(path); + srcExtCrud.delete(t); } - - logger.debug("Loaded " + srcTableExtMap.size() + " SourceTable EXD(s)"); } - private TableExtDesc reloadTableExtAt(String path) throws IOException { + private TableExtDesc convertOldTableExtToNewer(String resourceName) { ResourceStore store = getStore(); - String prj = TableExtDesc.parseResourcePath(path).getSecond(); - - TableExtDesc t = store.getResource(path, TableExtDesc.class, TABLE_EXT_SERIALIZER); - - if (t == null) { - return null; - } - - // convert old tableExt json to new one - if (t.getIdentity() == null) { - t = convertOldTableExtToNewer(path); - } - - t.init(prj); - - srcTableExtMap.putLocal(mapKey(t.getIdentity(), prj), t); - return t; - } - - private String mapKey(String identity, String prj) { - return prj == null ? identity : identity + "--" + prj; - } - - private TableExtDesc convertOldTableExtToNewer(String path) throws IOException { Map attrs = Maps.newHashMap(); - ResourceStore store = getStore(); - RawResource res = store.getResource(path); - - InputStream is = res.inputStream; - try { - attrs.putAll(JsonUtil.readValue(is, HashMap.class)); - } finally { - if (is != null) - is.close(); + RawResource res = store.getResource( + ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + resourceName + MetadataConstants.FILE_SURFIX); + + InputStream is = res.inputStream; + try { + attrs.putAll(JsonUtil.readValue(is, HashMap.class)); + } finally { + if (is != null) + is.close(); + } + } catch (IOException ex) { + throw new RuntimeException(ex); } String cardinality = attrs.get(MetadataConstants.TABLE_EXD_CARDINALITY); // parse table identity from file name - String file = path; - if (file.indexOf("/") > -1) { - file = file.substring(file.lastIndexOf("/") + 1); - } - String tableIdentity = file.substring(0, file.length() - MetadataConstants.FILE_SURFIX.length()).toUpperCase(); + String tableIdentity = TableDesc.parseResourcePath(resourceName).getFirst(); TableExtDesc result = new TableExtDesc(); result.setIdentity(tableIdentity); result.setUuid(UUID.randomUUID().toString()); @@ -440,66 +414,59 @@ private TableExtDesc convertOldTableExtToNewer(String path) throws IOException { return result; } - private void reloadAllExternalFilter() throws IOException { - ResourceStore store = getStore(); - logger.debug("Reloading ExternalFilter from folder " - + store.getReadableResourcePath(ResourceStore.EXTERNAL_FILTER_RESOURCE_ROOT)); + // ============================================================================ + // ExternalFilterDesc methods + // ============================================================================ - extFilterMap.clear(); + private void initExtFilter() throws IOException { + this.extFilterMap = new CaseInsensitiveStringCache<>(config, "external_filter"); + this.extFilterCrud = new CachedCrudAssist(getStore(), + ResourceStore.EXTERNAL_FILTER_RESOURCE_ROOT, ExternalFilterDesc.class, extFilterMap) { + @Override + protected ExternalFilterDesc initEntityAfterReload(ExternalFilterDesc t, String resourceName) { + return t; // noop + } + }; + extFilterCrud.reloadAll(); + Broadcaster.getInstance(config).registerListener(new ExtFilterSyncListener(), "external_filter"); + } - List paths = store.collectResourceRecursively(ResourceStore.EXTERNAL_FILTER_RESOURCE_ROOT, - MetadataConstants.FILE_SURFIX); - for (String path : paths) { - reloadExternalFilterAt(path); + private class ExtFilterSyncListener extends Broadcaster.Listener { + @Override + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { + try (AutoLock lock = extFilterMapLock.lockForWrite()) { + if (event == Event.DROP) + extFilterMap.removeLocal(cacheKey); + else + extFilterCrud.reloadQuietly(cacheKey); + } } - - logger.debug("Loaded " + extFilterMap.size() + " ExternalFilter(s)"); } - private void reloadAllSourceTable() throws IOException { - ResourceStore store = getStore(); - logger.debug("Reloading SourceTable from folder " - + store.getReadableResourcePath(ResourceStore.TABLE_RESOURCE_ROOT)); - - srcTableMap.clear(); - - List paths = store.collectResourceRecursively(ResourceStore.TABLE_RESOURCE_ROOT, - MetadataConstants.FILE_SURFIX); - for (String path : paths) { - reloadSourceTableAt(path); + public List listAllExternalFilters() { + try (AutoLock lock = extFilterMapLock.lockForRead()) { + return Lists.newArrayList(extFilterMap.values()); } - - logger.debug("Loaded " + srcTableMap.size() + " SourceTable(s)"); } - private TableDesc reloadSourceTableAt(String path) throws IOException { - ResourceStore store = getStore(); - String prj = TableDesc.parseResourcePath(path).getSecond(); - - TableDesc t = store.getResource(path, TableDesc.class, TABLE_SERIALIZER); - if (t == null) { - return null; + public ExternalFilterDesc getExtFilterDesc(String filterTableName) { + try (AutoLock lock = extFilterMapLock.lockForRead()) { + ExternalFilterDesc result = extFilterMap.get(filterTableName); + return result; } - t.init(prj); - - srcTableMap.putLocal(mapKey(t.getIdentity(), prj), t); - - return t; } - private ExternalFilterDesc reloadExternalFilterAt(String path) throws IOException { - ResourceStore store = getStore(); - ExternalFilterDesc t = store.getResource(path, ExternalFilterDesc.class, EXTERNAL_FILTER_DESC_SERIALIZER); - if (t == null) { - return null; + public void saveExternalFilter(ExternalFilterDesc desc) throws IOException { + try (AutoLock lock = extFilterMapLock.lockForWrite()) { + extFilterCrud.save(desc); } - extFilterMap.putLocal(t.getName(), t); - - return t; } - public void reloadExtFilter(String extFilterName) throws IOException { - reloadExternalFilterAt(ExternalFilterDesc.concatResourcePath(extFilterName)); + public void removeExternalFilter(String name) throws IOException { + try (AutoLock lock = extFilterMapLock.lockForWrite()) { + extFilterCrud.delete(name); + } } } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/TempStatementManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/TempStatementManager.java index 30ff934e9b1..970df0ccb1e 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/TempStatementManager.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/TempStatementManager.java @@ -23,11 +23,12 @@ import java.util.List; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.RootPersistentEntity; -import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.metadata.cachesync.Broadcaster; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,8 +38,6 @@ public class TempStatementManager { private static final Logger logger = LoggerFactory.getLogger(TempStatementManager.class); - public static final Serializer TEMP_STATEMENT_SERIALIZER = new JsonSerializer<>( - TempStatementEntity.class); public static TempStatementManager getInstance(KylinConfig config) { return config.getManager(TempStatementManager.class); @@ -52,73 +51,62 @@ static TempStatementManager newInstance(KylinConfig config) throws IOException { // ============================================================================ private KylinConfig config; - private CaseInsensitiveStringCache tempStatementMap; - - private TempStatementManager(KylinConfig config) throws IOException { - init(config); - } - - private void init(KylinConfig config) throws IOException { - this.config = config; - this.tempStatementMap = new CaseInsensitiveStringCache<>(config, "temp_statement"); + private CaseInsensitiveStringCache tmpStatMap; + private CachedCrudAssist crud; + private AutoReadWriteLock lock = new AutoReadWriteLock(); + + private TempStatementManager(KylinConfig cfg) throws IOException { + this.config = cfg; + this.tmpStatMap = new CaseInsensitiveStringCache<>(config, "temp_statement"); + this.crud = new CachedCrudAssist(getStore(), ResourceStore.TEMP_STATMENT_RESOURCE_ROOT, + TempStatementEntity.class, tmpStatMap) { + @Override + protected TempStatementEntity initEntityAfterReload(TempStatementEntity t, String resourceName) { + return t; // noop + } + }; - reloadAllTempStatement(); + crud.reloadAll(); // touch lower level metadata before registering my listener Broadcaster.getInstance(config).registerListener(new TempStatementSyncListener(), "temp_statement"); } - private void reloadAllTempStatement() throws IOException { - ResourceStore store = getStore(); - logger.debug("Reloading temp statement from folder " - + store.getReadableResourcePath(ResourceStore.TEMP_STATMENT_RESOURCE_ROOT)); - - tempStatementMap.clear(); + private class TempStatementSyncListener extends Broadcaster.Listener { - List paths = store.collectResourceRecursively(ResourceStore.TEMP_STATMENT_RESOURCE_ROOT, - MetadataConstants.FILE_SURFIX); - for (String path : paths) { - reloadTempStatementAt(path); + @Override + public void onEntityChange(Broadcaster broadcaster, String entity, Broadcaster.Event event, String cacheKey) + throws IOException { + try (AutoLock l = lock.lockForWrite()) { + if (event == Broadcaster.Event.DROP) + tmpStatMap.removeLocal(cacheKey); + else + crud.reloadQuietly(cacheKey); + } } - - logger.debug("Loaded " + tempStatementMap.size() + " Temp Statement(s)"); } - private TempStatementEntity reloadTempStatement(String statementId) throws IOException { - return reloadTempStatement(TempStatementEntity.DEFAULT_SESSION_ID, statementId); + public String getTempStatement(String statementId) { + return getTempStatement(TempStatementEntity.DEFAULT_SESSION_ID, statementId); } - private TempStatementEntity reloadTempStatement(String sessionId, String statementId) throws IOException { - return reloadTempStatementAt(TempStatementEntity.concatResourcePath(sessionId, statementId)); + public String getTempStatement(String sessionId, String statementId) { + TempStatementEntity entity = getTempStatEntity(sessionId, statementId); + return entity == null ? null : entity.statement; } - private TempStatementEntity reloadTempStatementAt(String path) throws IOException { - ResourceStore store = getStore(); - - TempStatementEntity s = store.getResource(path, TempStatementEntity.class, TEMP_STATEMENT_SERIALIZER); - if (s == null) { - return null; + public TempStatementEntity getTempStatEntity(String sessionId, String statementId) { + try (AutoLock l = lock.lockForRead()) { + return tmpStatMap.get(TempStatementEntity.resourceName(sessionId, statementId)); } - - tempStatementMap.putLocal(s.getMapKey(), s.getStatement()); - return s; - } - - public ResourceStore getStore() { - return ResourceStore.getStore(this.config); - } - - public String getTempStatement(String statementId) { - return getTempStatement(TempStatementEntity.DEFAULT_SESSION_ID, statementId); } - public String getTempStatement(String sessionId, String statementId) { - return tempStatementMap.get(TempStatementEntity.getMapKey(sessionId, statementId)); - } // for test - List listAllTempStatement() throws IOException { - reloadAllTempStatement(); - return new ArrayList<>(tempStatementMap.keySet()); + List reloadAllTempStatement() throws IOException { + try (AutoLock l = lock.lockForWrite()) { + crud.reloadAll(); + return new ArrayList<>(tmpStatMap.keySet()); + } } public void updateTempStatement(String statementId, String statement) throws IOException { @@ -126,15 +114,28 @@ public void updateTempStatement(String statementId, String statement) throws IOE } public void updateTempStatement(String sessionId, String statementId, String statement) throws IOException { - TempStatementEntity entity = new TempStatementEntity(sessionId, statementId, statement); - updateTempStatementWithRetry(entity, 0); - tempStatementMap.put(entity.getMapKey(), statement); + try (AutoLock l = lock.lockForWrite()) { + TempStatementEntity entity = new TempStatementEntity(sessionId, statementId, statement); + entity = prepareToOverwrite(entity, getTempStatEntity(sessionId, statementId)); + updateTempStatementWithRetry(entity, 0); + } } - public void updateTempStatementWithRetry(TempStatementEntity entity, int retry) throws IOException { - ResourceStore store = getStore(); + private TempStatementEntity prepareToOverwrite(TempStatementEntity entity, TempStatementEntity origin) { + if (origin == null) { + // create + entity.updateRandomUuid(); + } else { + // update + entity.setUuid(origin.getUuid()); + entity.setLastModified(origin.getLastModified()); + } + return entity; + } + + private void updateTempStatementWithRetry(TempStatementEntity entity, int retry) throws IOException { try { - store.putResource(entity.concatResourcePath(), entity, TEMP_STATEMENT_SERIALIZER); + crud.save(entity); } catch (IllegalStateException ise) { logger.warn("Write conflict to update temp statement" + entity.statementId + " at try " + retry + ", will retry..."); @@ -143,35 +144,27 @@ public void updateTempStatementWithRetry(TempStatementEntity entity, int retry) throw ise; } - TempStatementEntity reload = reloadTempStatement(entity.statementId); - reload.setStatement(entity.statement); - retry++; - updateTempStatementWithRetry(reload, retry); + TempStatementEntity reload = crud.reload(entity.resourceName()); + entity = prepareToOverwrite(entity, reload); + updateTempStatementWithRetry(entity, ++retry); } } + public void removeTempStatement(String statementId) throws IOException { removeTempStatement(TempStatementEntity.DEFAULT_SESSION_ID, statementId); } public void removeTempStatement(String session, String statementId) throws IOException { - ResourceStore store = getStore(); - store.deleteResource(TempStatementEntity.concatResourcePath(session, statementId)); - tempStatementMap.remove(TempStatementEntity.concatResourcePath(session, statementId)); + try (AutoLock l = lock.lockForWrite()) { + crud.delete(TempStatementEntity.resourceName(session, statementId)); + } } - private class TempStatementSyncListener extends Broadcaster.Listener { - - @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Broadcaster.Event event, String cacheKey) - throws IOException { - if (event == Broadcaster.Event.DROP) - tempStatementMap.removeLocal(cacheKey); - else - reloadTempStatementAt(TempStatementEntity.concatResourcePath(cacheKey)); - } + private ResourceStore getStore() { + return ResourceStore.getStore(this.config); } - @SuppressWarnings("serial") + @SuppressWarnings({ "serial", "unused" }) @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) private static class TempStatementEntity extends RootPersistentEntity { private static final String DEFAULT_SESSION_ID = "DEFAULT_SESSION"; @@ -217,10 +210,15 @@ public void setStatement(String statement) { * @return */ public String getMapKey() { + return resourceName(); + } + + @Override + public String resourceName() { return sessionId + "/" + statementId; } - public static String getMapKey(String sessionId, String statementId) { + public static String resourceName(String sessionId, String statementId) { return sessionId + "/" + statementId; } @@ -233,7 +231,8 @@ public static String concatResourcePath(String statementId) { } public static String concatResourcePath(String sessionId, String statementId) { - return ResourceStore.TEMP_STATMENT_RESOURCE_ROOT + "/" + sessionId + "/" + statementId + MetadataConstants.FILE_SURFIX; + return ResourceStore.TEMP_STATMENT_RESOURCE_ROOT + "/" + sessionId + "/" + statementId + + MetadataConstants.FILE_SURFIX; } } } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACL.java b/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACL.java index 22e55cc0f42..57ebb61314e 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACL.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACL.java @@ -48,6 +48,17 @@ public class TableACL extends RootPersistentEntity { @JsonProperty() private TableACLEntry groupTableBlackList = new TableACLEntry(); + private String project; + + void init(String project) { + this.project = project; + } + + @Override + public String resourceName() { + return project; + } + public Set getTableBlackList(String username, Set groups) { Set tableBlackList = new TreeSet<>(String.CASE_INSENSITIVE_ORDER); tableBlackList.addAll(userTableBlackList.getTableBlackList(username)); diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACLManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACLManager.java index 905fa16c547..163d340c74c 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACLManager.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/acl/TableACLManager.java @@ -19,13 +19,14 @@ package org.apache.kylin.metadata.acl; import java.io.IOException; -import java.util.List; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; -import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.metadata.cachesync.Broadcaster; +import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,9 +37,6 @@ public class TableACLManager { private static final Logger logger = LoggerFactory.getLogger(TableACLManager.class); - private static final Serializer TABLE_ACL_SERIALIZER = new JsonSerializer<>(TableACL.class); - private static final String DIR_PREFIX = "/table_acl/"; - public static TableACLManager getInstance(KylinConfig config) { return config.getManager(TableACLManager.class); } @@ -53,20 +51,36 @@ static TableACLManager newInstance(KylinConfig config) throws IOException { private KylinConfig config; // user ==> TableACL private CaseInsensitiveStringCache tableACLMap; + private CachedCrudAssist crud; + private AutoReadWriteLock lock = new AutoReadWriteLock(); public TableACLManager(KylinConfig config) throws IOException { logger.info("Initializing TableACLManager with config " + config); this.config = config; this.tableACLMap = new CaseInsensitiveStringCache<>(config, "table_acl"); - loadAllTableACL(); + this.crud = new CachedCrudAssist(getStore(), "/table_acl", "", TableACL.class, tableACLMap) { + @Override + protected TableACL initEntityAfterReload(TableACL acl, String resourceName) { + acl.init(resourceName); + return acl; + } + }; + + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new TableACLSyncListener(), "table_acl"); } private class TableACLSyncListener extends Broadcaster.Listener { @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Broadcaster.Event event, String cacheKey) throws IOException { - reloadTableACL(cacheKey); + public void onEntityChange(Broadcaster broadcaster, String entity, Broadcaster.Event event, String cacheKey) + throws IOException { + try (AutoLock l = lock.lockForWrite()) { + if (event == Event.DROP) + tableACLMap.removeLocal(cacheKey); + else + crud.reloadQuietly(cacheKey); + } broadcaster.notifyProjectACLUpdate(cacheKey); } } @@ -79,64 +93,57 @@ public ResourceStore getStore() { return ResourceStore.getStore(this.config); } - public TableACL getTableACLByCache(String project){ - TableACL tableACL = tableACLMap.get(project); - if (tableACL == null) { - return new TableACL(); + public TableACL getTableACLByCache(String project) { + try (AutoLock l = lock.lockForRead()) { + TableACL tableACL = tableACLMap.get(project); + if (tableACL == null) { + return newTableACL(project); + } + return tableACL; } - return tableACL; } - private void loadAllTableACL() throws IOException { - ResourceStore store = getStore(); - List paths = store.collectResourceRecursively("/table_acl", ""); - final int prefixLen = DIR_PREFIX.length(); - for (String path : paths) { - String project = path.substring(prefixLen, path.length()); - reloadTableACL(project); + public void addTableACL(String project, String name, String table, String type) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + TableACL tableACL = loadTableACL(project).add(name, table, type); + crud.save(tableACL); } - logger.info("Loading table ACL from folder " + store.getReadableResourcePath("/table_acl")); } - private void reloadTableACL(String project) throws IOException { - TableACL tableACLRecord = getTableACL(project); - tableACLMap.putLocal(project, tableACLRecord); + public void deleteTableACL(String project, String name, String table, String type) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + TableACL tableACL = loadTableACL(project).delete(name, table, type); + crud.save(tableACL); + } } - private TableACL getTableACL(String project) throws IOException { - String path = DIR_PREFIX + project; - TableACL tableACLRecord = getStore().getResource(path, TableACL.class, TABLE_ACL_SERIALIZER); - if (tableACLRecord == null) { - return new TableACL(); + public void deleteTableACL(String project, String name, String type) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + TableACL tableACL = loadTableACL(project).delete(name, type); + crud.save(tableACL); } - return tableACLRecord; } - public void addTableACL(String project, String name, String table, String type) throws IOException { - String path = DIR_PREFIX + project; - TableACL tableACL = getTableACL(project).add(name, table, type); - getStore().putResource(path, tableACL, System.currentTimeMillis(), TABLE_ACL_SERIALIZER); - tableACLMap.put(project, tableACL); + public void deleteTableACLByTbl(String project, String table) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + TableACL tableACL = loadTableACL(project).deleteByTbl(table); + crud.save(tableACL); + } } - public void deleteTableACL(String project, String name, String table, String type) throws IOException { - String path = DIR_PREFIX + project; - TableACL tableACL = getTableACL(project).delete(name, table, type); - getStore().putResource(path, tableACL, System.currentTimeMillis(), TABLE_ACL_SERIALIZER); - tableACLMap.put(project, tableACL); + private TableACL loadTableACL(String project) throws IOException { + TableACL acl = crud.reload(project); + if (acl == null) { + acl = newTableACL(project); + } + return acl; } - public void deleteTableACL(String project, String name, String type) throws IOException { - String path = DIR_PREFIX + project; - TableACL tableACL = getTableACL(project).delete(name, type); - getStore().putResource(path, tableACL, System.currentTimeMillis(), TABLE_ACL_SERIALIZER); - tableACLMap.put(project, tableACL); + private TableACL newTableACL(String project) { + TableACL acl = new TableACL(); + acl.updateRandomUuid(); + acl.init(project); + return acl; } - public void deleteTableACLByTbl(String project, String table) throws IOException { - String path = DIR_PREFIX + project; - TableACL tableACL = getTableACL(project).deleteByTbl(table); - getStore().putResource(path, tableACL, System.currentTimeMillis(), TABLE_ACL_SERIALIZER); - tableACLMap.put(project, tableACL); - } } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CachedCrudAssist.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CachedCrudAssist.java index 5c2ca17294c..ccc0a4a0df4 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CachedCrudAssist.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CachedCrudAssist.java @@ -26,6 +26,7 @@ import org.apache.kylin.common.persistence.RootPersistentEntity; import org.apache.kylin.common.persistence.Serializer; import org.apache.kylin.metadata.MetadataConstants; +import org.apache.kylin.metadata.model.DataModelDesc; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +39,7 @@ abstract public class CachedCrudAssist { final private ResourceStore store; final private Class entityType; final private String resRootPath; + final private String resPathSuffix; final private Serializer serializer; final private SingleValueCache cache; @@ -45,9 +47,15 @@ abstract public class CachedCrudAssist { public CachedCrudAssist(ResourceStore store, String resourceRootPath, Class entityType, SingleValueCache cache) { + this(store, resourceRootPath, MetadataConstants.FILE_SURFIX, entityType, cache); + } + + public CachedCrudAssist(ResourceStore store, String resourceRootPath, String resourcePathSuffix, + Class entityType, SingleValueCache cache) { this.store = store; this.entityType = entityType; this.resRootPath = resourceRootPath; + this.resPathSuffix = resourcePathSuffix; this.serializer = new JsonSerializer(entityType); this.cache = cache; @@ -57,12 +65,23 @@ public CachedCrudAssist(ResourceStore store, String resourceRootPath, Class e Preconditions.checkArgument(resRootPath.endsWith("/") == false); } + public Serializer getSerializer() { + return (Serializer) serializer; + } + public void setCheckOnWrite(boolean check) { this.checkCopyOnWrite = check; } private String resourcePath(String resourceName) { - return resRootPath + "/" + resourceName + MetadataConstants.FILE_SURFIX; + return resRootPath + "/" + resourceName + resPathSuffix; + } + + private String resourceName(String resourcePath) { + Preconditions.checkArgument(resourcePath.startsWith(resRootPath)); + Preconditions.checkArgument(resourcePath.endsWith(resPathSuffix)); + return resourcePath.substring(resRootPath.length() + 1, + resourcePath.length() - resPathSuffix.length()); } public void reloadAll() throws IOException { @@ -70,15 +89,16 @@ public void reloadAll() throws IOException { cache.clear(); - List paths = store.collectResourceRecursively(resRootPath, MetadataConstants.FILE_SURFIX); + List paths = store.collectResourceRecursively(resRootPath, resPathSuffix); for (String path : paths) { reloadQuietlyAt(path); } - logger.debug("Loaded " + cache.size() + " " + entityType.getName() + "(s)"); + logger.debug( + "Loaded " + cache.size() + " " + entityType.getName() + "(s) out of " + paths.size() + " resource"); } - public T reload(String resourceName) throws IOException { + public T reload(String resourceName) { return reloadAt(resourcePath(resourceName)); } @@ -95,25 +115,38 @@ private T reloadQuietlyAt(String path) { } } - public T reloadAt(String path) throws IOException { - T entity = store.getResource(path, entityType, serializer); - if (entity == null) { - logger.warn("No " + entityType.getName() + " found at " + path + ", returning null"); - return null; + public T reloadAt(String path) { + try { + T entity = store.getResource(path, entityType, serializer); + if (entity == null) { + logger.warn("No " + entityType.getName() + " found at " + path + ", returning null"); + cache.removeLocal(resourceName(path)); + return null; + } + + entity = initEntityAfterReload(entity, resourceName(path)); + + if (path.equals(resourcePath(entity.resourceName())) == false) + throw new IllegalStateException("The entity " + entity + " read from " + path + + " will save to a different path " + resourcePath(entity.resourceName())); + + cache.putLocal(entity.resourceName(), entity); + return entity; + } catch (Exception e) { + throw new IllegalStateException("Error loading " + entityType.getName() + " at " + path, e); } - - initEntityAfterReload(entity); - cache.putLocal(entity.resourceName(), entity); - return entity; } - abstract protected void initEntityAfterReload(T entity); + abstract protected T initEntityAfterReload(T entity, String resourceName); public T save(T entity) throws IOException { Preconditions.checkArgument(entity != null); + Preconditions.checkArgument(entity.getUuid() != null); Preconditions.checkArgument(entityType.isInstance(entity)); String resName = entity.resourceName(); + Preconditions.checkArgument(resName != null && resName.length() > 0); + if (checkCopyOnWrite) { if (cache.get(resName) == entity) { throw new IllegalStateException("Copy-on-write violation! The updating entity " + entity @@ -121,7 +154,10 @@ public T save(T entity) throws IOException { } } - store.putResource(resourcePath(resName), entity, serializer); + String path = resourcePath(resName); + logger.debug("Saving {} at {}", entityType.getName(), path); + + store.putResource(path, entity, serializer); cache.put(resName, entity); return entity; } @@ -132,7 +168,11 @@ public void delete(T entity) throws IOException { public void delete(String resName) throws IOException { Preconditions.checkArgument(resName != null); - store.deleteResource(resourcePath(resName)); + + String path = resourcePath(resName); + logger.debug("Deleting {} at {}", entityType.getName(), path); + + store.deleteResource(path); cache.remove(resName); } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java index 6e30ffb0114..bf0db7360e8 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java @@ -124,6 +124,11 @@ public DataModelDesc() { public KylinConfig getConfig() { return config; } + + @Override + public String resourceName() { + return name; + } public String getName() { return name; @@ -721,7 +726,7 @@ public String toString() { } public String getResourcePath() { - return concatResourcePath(name); + return concatResourcePath(resourceName()); } public static String concatResourcePath(String descName) { diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java index dc1ac442a81..c06a31bb701 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelManager.java @@ -24,15 +24,16 @@ import java.util.Map; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.common.util.ClassUtil; import org.apache.kylin.common.util.StringUtil; -import org.apache.kylin.metadata.MetadataConstants; import org.apache.kylin.metadata.TableMetadataManager; import org.apache.kylin.metadata.cachesync.Broadcaster; import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.apache.kylin.metadata.project.ProjectInstance; import org.apache.kylin.metadata.project.ProjectManager; @@ -65,48 +66,37 @@ static DataModelManager newInstance(KylinConfig conf) { // ============================================================================ private KylinConfig config; - private Serializer serializer; // name => DataModelDesc private CaseInsensitiveStringCache dataModelDescMap; + private CachedCrudAssist crud; + + // protects concurrent operations around the cached map, to avoid for example + // writing an entity in the middle of reloading it (dirty read) + private AutoReadWriteLock modelMapLock = new AutoReadWriteLock(); public DataModelManager(KylinConfig config) throws IOException { init(config); } - public KylinConfig getConfig() { - return config; - } - - public ResourceStore getStore() { - return ResourceStore.getStore(this.config); - } - - public Serializer getDataModelSerializer() { - if (serializer == null) { - try { - String cls = StringUtil.noBlank(config.getDataModelImpl(), DataModelDesc.class.getName()); - Class clz = ClassUtil.forName(cls, DataModelDesc.class); - serializer = new JsonSerializer(clz); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - return serializer; - } - - public List listDataModels() { - return Lists.newArrayList(dataModelDescMap.values()); - } - - protected void init(KylinConfig config) throws IOException { - this.config = config; + protected void init(KylinConfig cfg) throws IOException { + this.config = cfg; this.dataModelDescMap = new CaseInsensitiveStringCache<>(config, "data_model"); + this.crud = new CachedCrudAssist(getStore(), ResourceStore.DATA_MODEL_DESC_RESOURCE_ROOT, + getDataModelImplClass(), dataModelDescMap) { + @Override + protected DataModelDesc initEntityAfterReload(DataModelDesc model, String resourceName) { + String prj = ProjectManager.getInstance(config).getProjectOfModel(model.getName()).getName(); + if (!model.isDraft()) { + model.init(config, getAllTablesMap(prj), listDataModels(), true); + } + return model; + } + }; // touch lower level metadata before registering model listener TableMetadataManager.getInstance(config); - - reloadAllDataModel(); + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new DataModelSyncListener(), "data_model"); } @@ -117,18 +107,22 @@ public void onProjectSchemaChange(Broadcaster broadcaster, String project) throw //clean up the current project's table desc TableMetadataManager.getInstance(config).resetProjectSpecificTableDesc(project); - for (String model : ProjectManager.getInstance(config).getProject(project).getModels()) { - reloadDataModelDescAt(DataModelDesc.concatResourcePath(model)); + try (AutoLock lock = modelMapLock.lockForWrite()) { + for (String model : ProjectManager.getInstance(config).getProject(project).getModels()) { + crud.reloadQuietly(model); + } } } @Override public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException { - if (event == Event.DROP) - dataModelDescMap.removeLocal(cacheKey); - else - reloadDataModelDescAt(DataModelDesc.concatResourcePath(cacheKey)); + try (AutoLock lock = modelMapLock.lockForWrite()) { + if (event == Event.DROP) + dataModelDescMap.removeLocal(cacheKey); + else + crud.reloadQuietly(cacheKey); + } for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByModel(cacheKey)) { broadcaster.notifyProjectSchemaUpdate(prj.getName()); @@ -136,141 +130,143 @@ public void onEntityChange(Broadcaster broadcaster, String entity, Event event, } } - public DataModelDesc getDataModelDesc(String name) { - return dataModelDescMap.get(name); + private Class getDataModelImplClass() { + try { + String cls = StringUtil.noBlank(config.getDataModelImpl(), DataModelDesc.class.getName()); + Class clz = ClassUtil.forName(cls, DataModelDesc.class); + return (Class) clz; + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } } - public List getModels() { - return new ArrayList<>(dataModelDescMap.values()); + public KylinConfig getConfig() { + return config; } - public List getModels(String projectName) { - ProjectInstance projectInstance = ProjectManager.getInstance(config).getProject(projectName); - ArrayList ret = new ArrayList<>(); - - if (projectInstance != null && projectInstance.getModels() != null) { - for (String modelName : projectInstance.getModels()) { - DataModelDesc model = getDataModelDesc(modelName); - if (null != model) { - ret.add(model); - } else { - logger.error("Failed to load model " + modelName); - } - } - } + public ResourceStore getStore() { + return ResourceStore.getStore(this.config); + } - return ret; + // for test mostly + public Serializer getDataModelSerializer() { + return crud.getSerializer(); } - // within a project, find models that use the specified table - public List getModelsUsingTable(TableDesc table, String project) throws IOException { - List models = new ArrayList<>(); - for (DataModelDesc modelDesc : getModels(project)) { - if (modelDesc.containsTable(table)) - models.add(modelDesc.getName()); + public List listDataModels() { + try (AutoLock lock = modelMapLock.lockForRead()) { + return Lists.newArrayList(dataModelDescMap.values()); } - return models; } - public boolean isTableInAnyModel(TableDesc table) { - for (DataModelDesc modelDesc : getModels()) { - if (modelDesc.containsTable(table)) - return true; + public DataModelDesc getDataModelDesc(String name) { + try (AutoLock lock = modelMapLock.lockForRead()) { + return dataModelDescMap.get(name); } - return false; } - private void reloadAllDataModel() throws IOException { - ResourceStore store = getStore(); - logger.debug("Reloading DataModel from folder " - + store.getReadableResourcePath(ResourceStore.DATA_MODEL_DESC_RESOURCE_ROOT)); + public List getModels() { + try (AutoLock lock = modelMapLock.lockForRead()) { + return new ArrayList<>(dataModelDescMap.values()); + } + } - dataModelDescMap.clear(); + public List getModels(String projectName) { + try (AutoLock lock = modelMapLock.lockForRead()) { + ProjectInstance projectInstance = ProjectManager.getInstance(config).getProject(projectName); + ArrayList ret = new ArrayList<>(); + + if (projectInstance != null && projectInstance.getModels() != null) { + for (String modelName : projectInstance.getModels()) { + DataModelDesc model = getDataModelDesc(modelName); + if (null != model) { + ret.add(model); + } else { + logger.error("Failed to load model " + modelName); + } + } + } - List paths = store.collectResourceRecursively(ResourceStore.DATA_MODEL_DESC_RESOURCE_ROOT, - MetadataConstants.FILE_SURFIX); - for (String path : paths) { + return ret; + } + } - try { - logger.info("Reloading data model at " + path); - reloadDataModelDescAt(path); - } catch (IllegalStateException e) { - logger.error("Error to load DataModel at " + path, e); - continue; + // within a project, find models that use the specified table + public List getModelsUsingTable(TableDesc table, String project) throws IOException { + try (AutoLock lock = modelMapLock.lockForRead()) { + List models = new ArrayList<>(); + for (DataModelDesc modelDesc : getModels(project)) { + if (modelDesc.containsTable(table)) + models.add(modelDesc.getName()); } + return models; } - - logger.debug("Loaded " + dataModelDescMap.size() + " DataModel(s)"); } - public DataModelDesc reloadDataModelDescAt(String path) { - ResourceStore store = getStore(); - try { - DataModelDesc dataModelDesc = store.getResource(path, DataModelDesc.class, getDataModelSerializer()); - String prj = ProjectManager.getInstance(config).getProjectOfModel(dataModelDesc.getName()).getName(); - - if (!dataModelDesc.isDraft()) { - dataModelDesc.init(config, this.getAllTablesMap(prj), listDataModels(), true); + public boolean isTableInAnyModel(TableDesc table) { + try (AutoLock lock = modelMapLock.lockForRead()) { + for (DataModelDesc modelDesc : getModels()) { + if (modelDesc.containsTable(table)) + return true; } - - dataModelDescMap.putLocal(dataModelDesc.getName(), dataModelDesc); - return dataModelDesc; - } catch (Exception e) { - throw new IllegalStateException("Error to load " + path, e); + return false; } } - // sync on update - public DataModelDesc dropModel(DataModelDesc desc) throws IOException { - logger.info("Dropping model '" + desc.getName() + "'"); - ResourceStore store = getStore(); - store.deleteResource(desc.getResourcePath()); - // delete model from project - ProjectManager.getInstance(config).removeModelFromProjects(desc.getName()); - // clean model cache - this.afterModelDropped(desc); - return desc; + public DataModelDesc reloadDataModel(String modelName) { + try (AutoLock lock = modelMapLock.lockForWrite()) { + return crud.reload(modelName); + } } - private void afterModelDropped(DataModelDesc desc) { - dataModelDescMap.remove(desc.getName()); + public DataModelDesc dropModel(DataModelDesc desc) throws IOException { + try (AutoLock lock = modelMapLock.lockForWrite()) { + crud.delete(desc); + // delete model from project + ProjectManager.getInstance(config).removeModelFromProjects(desc.getName()); + return desc; + } } public DataModelDesc createDataModelDesc(DataModelDesc desc, String projectName, String owner) throws IOException { - String name = desc.getName(); - if (dataModelDescMap.containsKey(name)) - throw new IllegalArgumentException("DataModelDesc '" + name + "' already exists"); + try (AutoLock lock = modelMapLock.lockForWrite()) { + String name = desc.getName(); + if (dataModelDescMap.containsKey(name)) + throw new IllegalArgumentException("DataModelDesc '" + name + "' already exists"); - ProjectManager prjMgr = ProjectManager.getInstance(config); - ProjectInstance prj = prjMgr.getProject(projectName); - if (prj.containsModel(name)) - throw new IllegalStateException("project " + projectName + " already contains model " + name); + ProjectManager prjMgr = ProjectManager.getInstance(config); + ProjectInstance prj = prjMgr.getProject(projectName); + if (prj.containsModel(name)) + throw new IllegalStateException("project " + projectName + " already contains model " + name); - try { - // Temporarily register model under project, because we want to - // update project formally after model is saved. - prj.getModels().add(name); + try { + // Temporarily register model under project, because we want to + // update project formally after model is saved. + prj.getModels().add(name); - desc.setOwner(owner); - desc = saveDataModelDesc(desc); + desc.setOwner(owner); + desc = saveDataModelDesc(desc); - } finally { - prj.getModels().remove(name); - } + } finally { + prj.getModels().remove(name); + } - // now that model is saved, update project formally - prjMgr.addModelToProject(name, projectName); + // now that model is saved, update project formally + prjMgr.addModelToProject(name, projectName); - return desc; + return desc; + } } public DataModelDesc updateDataModelDesc(DataModelDesc desc) throws IOException { - String name = desc.getName(); - if (!dataModelDescMap.containsKey(name)) { - throw new IllegalArgumentException("DataModelDesc '" + name + "' does not exist."); - } + try (AutoLock lock = modelMapLock.lockForWrite()) { + String name = desc.getName(); + if (!dataModelDescMap.containsKey(name)) { + throw new IllegalArgumentException("DataModelDesc '" + name + "' does not exist."); + } - return saveDataModelDesc(desc); + return saveDataModelDesc(desc); + } } private DataModelDesc saveDataModelDesc(DataModelDesc dataModelDesc) throws IOException { @@ -280,9 +276,7 @@ private DataModelDesc saveDataModelDesc(DataModelDesc dataModelDesc) throws IOEx if (!dataModelDesc.isDraft()) dataModelDesc.init(config, this.getAllTablesMap(prj), listDataModels(), true); - String path = dataModelDesc.getResourcePath(); - getStore().putResource(path, dataModelDesc, getDataModelSerializer()); - dataModelDescMap.put(dataModelDesc.getName(), dataModelDesc); + crud.save(dataModelDesc); return dataModelDesc; } @@ -290,4 +284,5 @@ private DataModelDesc saveDataModelDesc(DataModelDesc dataModelDesc) throws IOEx private Map getAllTablesMap(String prj) { return TableMetadataManager.getInstance(config).getAllTablesMap(prj); } + } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExternalFilterDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExternalFilterDesc.java index 115b1542281..35018c75c20 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExternalFilterDesc.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ExternalFilterDesc.java @@ -58,6 +58,11 @@ public static String concatResourcePath(String name) { // ============================================================================ + @Override + public String resourceName() { + return name; + } + public String getFilterResourceIdentifier() { return filterResourceIdentifier; } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java index 3b779db67dc..e456d9af0d2 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java @@ -41,11 +41,49 @@ @SuppressWarnings("serial") @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) public class TableDesc extends RootPersistentEntity implements ISourceAware { + @SuppressWarnings("unused") private static final Logger logger = LoggerFactory.getLogger(TableDesc.class); private static final String TABLE_TYPE_VIRTUAL_VIEW = "VIRTUAL_VIEW"; private static final String materializedTableNamePrefix = "kylin_intermediate_"; + public static String concatRawResourcePath(String nameOnPath) { + return ResourceStore.TABLE_RESOURCE_ROOT + "/" + nameOnPath + ".json"; + } + + public static String makeResourceName(String tableIdentity, String prj) { + return prj == null ? tableIdentity : tableIdentity + "--" + prj; + } + + // this method should only used for getting dest path when copying from src to dest. + // if you want to get table's src path, use getResourcePath() instead. + private static String concatResourcePath(String tableIdentity, String prj) { + return concatRawResourcePath(makeResourceName(tableIdentity, prj)); + } + + // returns + public static Pair parseResourcePath(String path) { + if (path.endsWith(".json")) + path = path.substring(0, path.length() - ".json".length()); + + int cut = path.lastIndexOf("/"); + if (cut >= 0) + path = path.substring(cut + 1); + + String table, prj; + int dash = path.indexOf("--"); + if (dash >= 0) { + table = path.substring(0, dash); + prj = path.substring(dash + 2); + } else { + table = path; + prj = null; + } + return Pair.newPair(table, prj); + } + + // ============================================================================ + @JsonProperty("name") private String name; @JsonProperty("columns") @@ -87,6 +125,11 @@ public TableDesc(TableDesc other) { this.identity = other.identity; } + @Override + public String resourceName() { + return makeResourceName(getIdentity(), getProject()); + } + public TableDesc appendColumns(ColumnDesc[] computedColumns, boolean makeCopy) { if (computedColumns == null || computedColumns.length == 0) { return this; @@ -170,42 +213,6 @@ public boolean isView() { return TABLE_TYPE_VIRTUAL_VIEW.equals(tableType); } - public static String concatRawResourcePath(String nameOnPath) { - return ResourceStore.TABLE_RESOURCE_ROOT + "/" + nameOnPath + ".json"; - } - - // this method should only used for getting dest path when copying from src to dest. - // if you want to get table's src path, use getResourcePath() instead. - private static String concatResourcePath(String tableIdentity, String prj) { - if (prj == null || prj.isEmpty()) - return ResourceStore.TABLE_RESOURCE_ROOT + "/" + tableIdentity + ".json"; - else - return ResourceStore.TABLE_RESOURCE_ROOT + "/" + tableIdentity + "--" + prj + ".json"; - } - - // returns - public static Pair parseResourcePath(String path) { - if (path.endsWith(".json")) - path = path.substring(0, path.length() - ".json".length()); - - int cut = path.lastIndexOf("/"); - if (cut >= 0) - path = path.substring(cut + 1); - - String table, prj; - int dash = path.indexOf("--"); - if (dash >= 0) { - table = path.substring(0, dash); - prj = path.substring(dash + 2); - } else { - table = path; - prj = null; - } - return Pair.newPair(table, prj); - } - - // ============================================================================ - public boolean isBorrowedFromGlobal() { return isBorrowedFromGlobal; } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java index 1b248e5c1c0..28ea2382e45 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableExtDesc.java @@ -38,6 +38,21 @@ @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) public class TableExtDesc extends RootPersistentEntity { + public static String concatRawResourcePath(String nameOnPath) { + return ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + nameOnPath + ".json"; + } + + public static String concatResourcePath(String tableIdentity, String prj) { + return concatRawResourcePath(TableDesc.makeResourceName(tableIdentity, prj)); + } + + // returns + public static Pair parseResourcePath(String path) { + return TableDesc.parseResourcePath(path); + } + + // ============================================================================ + @JsonProperty("table_name") private String tableIdentity; @JsonProperty("last_build_job_id") @@ -65,24 +80,13 @@ public class TableExtDesc extends RootPersistentEntity { public TableExtDesc() { } - public String getResourcePath() { - return concatResourcePath(getIdentity(), project); - } - - public static String concatRawResourcePath(String nameOnPath) { - return ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + nameOnPath + ".json"; - } - - public static String concatResourcePath(String tableIdentity, String prj) { - if (prj == null) - return ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + tableIdentity + ".json"; - else - return ResourceStore.TABLE_EXD_RESOURCE_ROOT + "/" + tableIdentity + "--" + prj + ".json"; + @Override + public String resourceName() { + return TableDesc.makeResourceName(getIdentity(), getProject()); } - - // returns - public static Pair parseResourcePath(String path) { - return TableDesc.parseResourcePath(path); + + public String getResourcePath() { + return concatResourcePath(getIdentity(), getProject()); } public String getProject() { diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java index 7b1f8403676..0029de26597 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectInstance.java @@ -88,7 +88,7 @@ public class ProjectInstance extends RootPersistentEntity { private LinkedHashMap overrideKylinProps; public String getResourcePath() { - return concatResourcePath(name); + return concatResourcePath(resourceName()); } public static String concatResourcePath(String projectName) { diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java index 4622f3578e5..1c0254ec2e5 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java @@ -70,12 +70,11 @@ static ProjectManager newInstance(KylinConfig config) throws IOException { // project name => ProjrectInstance private CaseInsensitiveStringCache projectMap; + private CachedCrudAssist crud; - // protects concurrent operations around the projectMap, to avoid for example - // writing a project in the middle of reloading it (dirty read) + // protects concurrent operations around the cached map, to avoid for example + // writing an entity in the middle of reloading it (dirty read) private AutoReadWriteLock prjMapLock = new AutoReadWriteLock(); - - private CachedCrudAssist crud; private ProjectManager(KylinConfig config) throws IOException { logger.info("Initializing ProjectManager with metadata url " + config); @@ -85,13 +84,14 @@ private ProjectManager(KylinConfig config) throws IOException { this.crud = new CachedCrudAssist(getStore(), ResourceStore.PROJECT_RESOURCE_ROOT, ProjectInstance.class, projectMap) { @Override - protected void initEntityAfterReload(ProjectInstance prj) { + protected ProjectInstance initEntityAfterReload(ProjectInstance prj, String resourceName) { prj.init(); + return prj; } }; + // touch lower level metadata before registering my listener crud.reloadAll(); - Broadcaster.getInstance(config).registerListener(new ProjectSyncListener(), "project"); } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingConfig.java b/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingConfig.java index 9fd6edeae91..335d3c8a00d 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingConfig.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingConfig.java @@ -34,6 +34,7 @@ /** */ +@SuppressWarnings("serial") @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) public class StreamingConfig extends RootPersistentEntity { @@ -47,6 +48,11 @@ public class StreamingConfig extends RootPersistentEntity { @JsonProperty("type") private String type = STREAMING_TYPE_KAFKA; + @Override + public String resourceName() { + return name; + } + public String getType() { return type; } diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingManager.java index b5d7e37742d..d72058565c5 100644 --- a/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingManager.java +++ b/core-metadata/src/main/java/org/apache/kylin/metadata/streaming/StreamingManager.java @@ -24,12 +24,12 @@ import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; -import org.apache.kylin.common.persistence.Serializer; -import org.apache.kylin.metadata.MetadataConstants; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.metadata.cachesync.Broadcaster; import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,10 +38,9 @@ */ public class StreamingManager { + @SuppressWarnings("unused") private static final Logger logger = LoggerFactory.getLogger(StreamingManager.class); - public static final Serializer STREAMING_SERIALIZER = new JsonSerializer(StreamingConfig.class); - public static StreamingManager getInstance(KylinConfig config) { return config.getManager(StreamingManager.class); } @@ -52,29 +51,43 @@ static StreamingManager newInstance(KylinConfig config) throws IOException { } // ============================================================================ - + private KylinConfig config; // name ==> StreamingConfig private CaseInsensitiveStringCache streamingMap; + private CachedCrudAssist crud; + private AutoReadWriteLock lock = new AutoReadWriteLock(); private StreamingManager(KylinConfig config) throws IOException { this.config = config; this.streamingMap = new CaseInsensitiveStringCache(config, "streaming"); - + this.crud = new CachedCrudAssist(getStore(), ResourceStore.STREAMING_RESOURCE_ROOT, + StreamingConfig.class, streamingMap) { + @Override + protected StreamingConfig initEntityAfterReload(StreamingConfig t, String resourceName) { + return t; // noop + } + }; + // touch lower level metadata before registering my listener - reloadAllStreaming(); + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new StreamingSyncListener(), "streaming"); } private class StreamingSyncListener extends Broadcaster.Listener { @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException { - if (event == Event.DROP) - removeStreamingLocal(cacheKey); - else - reloadStreamingConfigLocal(cacheKey); + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { + String streamingName = cacheKey; + + try (AutoLock l = lock.lockForWrite()) { + if (event == Event.DROP) + streamingMap.removeLocal(streamingName); + else + crud.reloadQuietly(streamingName); + } } } @@ -83,129 +96,57 @@ private ResourceStore getStore() { } public StreamingConfig getStreamingConfig(String name) { - return streamingMap.get(name); + try (AutoLock l = lock.lockForRead()) { + return streamingMap.get(name); + } } public List listAllStreaming() { - return new ArrayList<>(streamingMap.values()); - } - - /** - * Reload StreamingConfig from resource store It will be triggered by an desc - * update event. - * - * @param name - * @throws IOException - */ - public StreamingConfig reloadStreamingConfigLocal(String name) throws IOException { - - // Save Source - String path = StreamingConfig.concatResourcePath(name); - - // Reload the StreamingConfig - StreamingConfig ndesc = loadStreamingConfigAt(path); - - // Here replace the old one - streamingMap.putLocal(ndesc.getName(), ndesc); - return ndesc; - } - - // remove streamingConfig - public void removeStreamingConfig(StreamingConfig streamingConfig) throws IOException { - String path = streamingConfig.getResourcePath(); - getStore().deleteResource(path); - streamingMap.remove(streamingConfig.getName()); - } - - public StreamingConfig getConfig(String name) { - name = name.toUpperCase(); - return streamingMap.get(name); - } - - public void removeStreamingLocal(String streamingName) { - streamingMap.removeLocal(streamingName); - } - - /** - * Update CubeDesc with the input. Broadcast the event into cluster - * - * @param desc - * @return - * @throws IOException - */ - public StreamingConfig updateStreamingConfig(StreamingConfig desc) throws IOException { - // Validate CubeDesc - if (desc.getUuid() == null || desc.getName() == null) { - throw new IllegalArgumentException("SteamingConfig Illegal."); - } - String name = desc.getName(); - if (!streamingMap.containsKey(name)) { - throw new IllegalArgumentException("StreamingConfig '" + name + "' does not exist."); + try (AutoLock l = lock.lockForRead()) { + return new ArrayList<>(streamingMap.values()); } - - // Save Source - String path = desc.getResourcePath(); - getStore().putResource(path, desc, STREAMING_SERIALIZER); - - // Reload the StreamingConfig - StreamingConfig ndesc = loadStreamingConfigAt(path); - // Here replace the old one - streamingMap.put(ndesc.getName(), desc); - - return ndesc; } - - public StreamingConfig saveStreamingConfig(StreamingConfig streamingConfig) throws IOException { - if (streamingConfig == null || StringUtils.isEmpty(streamingConfig.getName())) { - throw new IllegalArgumentException(); - } - - if (streamingMap.containsKey(streamingConfig.getName())) - throw new IllegalArgumentException("StreamingConfig '" + streamingConfig.getName() + "' already exists"); - - String path = StreamingConfig.concatResourcePath(streamingConfig.getName()); - getStore().putResource(path, streamingConfig, StreamingConfig.SERIALIZER); - streamingMap.put(streamingConfig.getName(), streamingConfig); - return streamingConfig; + + // for test + List reloadAll() throws IOException { + try (AutoLock l = lock.lockForWrite()) { + crud.reloadAll(); + return listAllStreaming(); + } } - private StreamingConfig loadStreamingConfigAt(String path) throws IOException { - ResourceStore store = getStore(); - StreamingConfig streamingDesc = store.getResource(path, StreamingConfig.class, STREAMING_SERIALIZER); + public StreamingConfig createStreamingConfig(StreamingConfig streamingConfig) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + if (streamingConfig == null || StringUtils.isEmpty(streamingConfig.getName())) { + throw new IllegalArgumentException(); + } + if (streamingMap.containsKey(streamingConfig.resourceName())) + throw new IllegalArgumentException( + "StreamingConfig '" + streamingConfig.getName() + "' already exists"); - if (StringUtils.isBlank(streamingDesc.getName())) { - throw new IllegalStateException("StreamingConfig name must not be blank"); + streamingConfig.updateRandomUuid(); + + return crud.save(streamingConfig); } - return streamingDesc; } - private void reloadAllStreaming() throws IOException { - ResourceStore store = getStore(); - logger.info("Reloading Streaming Metadata from folder " + store.getReadableResourcePath(ResourceStore.STREAMING_RESOURCE_ROOT)); - - streamingMap.clear(); - - List paths = store.collectResourceRecursively(ResourceStore.STREAMING_RESOURCE_ROOT, MetadataConstants.FILE_SURFIX); - for (String path : paths) { - StreamingConfig streamingConfig; - try { - streamingConfig = loadStreamingConfigAt(path); - } catch (Exception e) { - logger.error("Error loading streaming desc " + path, e); - continue; - } - if (path.equals(streamingConfig.getResourcePath()) == false) { - logger.error("Skip suspicious desc at " + path + ", " + streamingConfig + " should be at " + streamingConfig.getResourcePath()); - continue; + public StreamingConfig updateStreamingConfig(StreamingConfig desc) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + if (desc.getUuid() == null || desc.getName() == null) { + throw new IllegalArgumentException("SteamingConfig Illegal."); } - if (streamingMap.containsKey(streamingConfig.getName())) { - logger.error("Dup StreamingConfig name '" + streamingConfig.getName() + "' on path " + path); - continue; + if (!streamingMap.containsKey(desc.resourceName())) { + throw new IllegalArgumentException("StreamingConfig '" + desc.getName() + "' does not exist."); } - streamingMap.putLocal(streamingConfig.getName(), streamingConfig); + return crud.save(desc); } + } - logger.debug("Loaded " + streamingMap.size() + " StreamingConfig(s)"); + public void removeStreamingConfig(StreamingConfig streamingConfig) throws IOException { + try (AutoLock l = lock.lockForWrite()) { + crud.delete(streamingConfig); + } } + } diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/TempStatementManagerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/TempStatementManagerTest.java index f2baf2900f6..663816c5ccc 100644 --- a/core-metadata/src/test/java/org/apache/kylin/metadata/TempStatementManagerTest.java +++ b/core-metadata/src/test/java/org/apache/kylin/metadata/TempStatementManagerTest.java @@ -46,21 +46,21 @@ public void testGetInstance() { public void testAddTempStatement() throws IOException { TempStatementManager manager = TempStatementManager.getInstance(getTestConfig()); manager.updateTempStatement("temp_table3", "AAAAA"); - Assert.assertEquals(3, manager.listAllTempStatement().size()); + Assert.assertEquals(3, manager.reloadAllTempStatement().size()); } @Test public void testRemoveTempStatement() throws IOException { TempStatementManager manager = TempStatementManager.getInstance(getTestConfig()); manager.removeTempStatement("temp_table1"); - Assert.assertEquals(1, manager.listAllTempStatement().size()); + Assert.assertEquals(1, manager.reloadAllTempStatement().size()); } @Test public void testUpdateTempStatement() throws IOException { TempStatementManager manager = TempStatementManager.getInstance(getTestConfig()); manager.updateTempStatement("temp_table1", "AAAAA"); - Assert.assertEquals(2, manager.listAllTempStatement().size()); + Assert.assertEquals(2, manager.reloadAllTempStatement().size()); Assert.assertEquals("AAAAA", manager.getTempStatement("temp_table1")); } } diff --git a/core-metadata/src/test/java/org/apache/kylin/metadata/streaming/StreamingManagerTest.java b/core-metadata/src/test/java/org/apache/kylin/metadata/streaming/StreamingManagerTest.java new file mode 100644 index 00000000000..798deb04841 --- /dev/null +++ b/core-metadata/src/test/java/org/apache/kylin/metadata/streaming/StreamingManagerTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ + +package org.apache.kylin.metadata.streaming; + +import java.io.IOException; +import java.util.List; + +import org.apache.kylin.common.util.LocalFileMetadataTestCase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class StreamingManagerTest extends LocalFileMetadataTestCase { + + @Before + public void setUp() throws Exception { + this.createTestMetadata(); + } + + @After + public void after() throws Exception { + this.cleanupTestMetadata(); + } + + @Test + public void testBasics() throws IOException { + StreamingManager mgr = StreamingManager.getInstance(getTestConfig()); + List origin = mgr.listAllStreaming(); + + // test create + { + StreamingConfig streamingConfig = new StreamingConfig(); + streamingConfig.setName("name for test"); + streamingConfig.setType("type for test"); + mgr.createStreamingConfig(streamingConfig); + List reloadAll = mgr.reloadAll(); + Assert.assertTrue(origin.size() + 1 == reloadAll.size()); + } + + // test update + { + StreamingConfig streamingConfig = mgr.getStreamingConfig("name for test"); + streamingConfig.setType("updated type"); + mgr.updateStreamingConfig(streamingConfig); + List reloadAll = mgr.reloadAll(); + Assert.assertTrue(origin.size() + 1 == reloadAll.size()); + streamingConfig = mgr.getStreamingConfig("name for test"); + Assert.assertEquals("updated type", streamingConfig.getType()); + } + } +} diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java index 14ef5243e85..9fbb7f3fbe7 100644 --- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java +++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java @@ -50,16 +50,27 @@ @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) public class HybridInstance extends RootPersistentEntity implements IRealization { + private final static Logger logger = LoggerFactory.getLogger(HybridInstance.class); + + public static HybridInstance create(KylinConfig config, String name, List realizationEntries) { + HybridInstance hybridInstance = new HybridInstance(); + + hybridInstance.setConfig(config); + hybridInstance.setName(name); + hybridInstance.setRealizationEntries(realizationEntries); + hybridInstance.updateRandomUuid(); + + return hybridInstance; + } + + // ============================================================================ + @JsonIgnore private KylinConfig config; @JsonProperty("name") private String name; - public void setRealizationEntries(List realizationEntries) { - this.realizationEntries = realizationEntries; - } - @JsonProperty("realizations") private List realizationEntries; @@ -75,21 +86,17 @@ public void setRealizationEntries(List realizationEntries) { private long dateRangeEnd; private boolean isReady = false; - private final static Logger logger = LoggerFactory.getLogger(HybridInstance.class); - + @Override + public String resourceName() { + return name; + } + public List getRealizationEntries() { return realizationEntries; } - public static HybridInstance create(KylinConfig config, String name, List realizationEntries) { - HybridInstance hybridInstance = new HybridInstance(); - - hybridInstance.setConfig(config); - hybridInstance.setName(name); - hybridInstance.setRealizationEntries(realizationEntries); - hybridInstance.updateRandomUuid(); - - return hybridInstance; + public void setRealizationEntries(List realizationEntries) { + this.realizationEntries = realizationEntries; } private void init() { diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java index 37f4aff411a..1e56c73c254 100644 --- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java +++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java @@ -21,13 +21,15 @@ import java.util.Collection; import java.util.List; -import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.Serializer; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.metadata.cachesync.Broadcaster; import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.apache.kylin.metadata.project.ProjectInstance; import org.apache.kylin.metadata.project.ProjectManager; @@ -43,9 +45,10 @@ /** */ public class HybridManager implements IRealizationProvider { - public static final Serializer HYBRID_SERIALIZER = new JsonSerializer(HybridInstance.class); private static final Logger logger = LoggerFactory.getLogger(HybridManager.class); + + public static final Serializer HYBRID_SERIALIZER = new JsonSerializer<>(HybridInstance.class); public static HybridManager getInstance(KylinConfig config) { return config.getManager(HybridManager.class); @@ -61,14 +64,24 @@ static HybridManager newInstance(KylinConfig config) throws IOException { private KylinConfig config; private CaseInsensitiveStringCache hybridMap; + private CachedCrudAssist crud; + private AutoReadWriteLock lock = new AutoReadWriteLock(); - private HybridManager(KylinConfig config) throws IOException { - logger.info("Initializing HybridManager with config " + config); - this.config = config; + private HybridManager(KylinConfig cfg) throws IOException { + logger.info("Initializing HybridManager with config " + cfg); + this.config = cfg; this.hybridMap = new CaseInsensitiveStringCache(config, "hybrid"); + this.crud = new CachedCrudAssist(getStore(), ResourceStore.HYBRID_RESOURCE_ROOT, + HybridInstance.class, hybridMap) { + @Override + protected HybridInstance initEntityAfterReload(HybridInstance hybridInstance, String resourceName) { + hybridInstance.setConfig(config); + return hybridInstance; // noop + } + }; // touch lower level metadata before registering my listener - reloadAllHybridInstance(); + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new HybridSyncListener(), "hybrid", "cube"); } @@ -76,89 +89,57 @@ private class HybridSyncListener extends Broadcaster.Listener { @Override public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException { - for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) { - if (real instanceof HybridInstance) { - reloadHybridInstance(real.getName()); + try (AutoLock l = lock.lockForWrite()) { + for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) { + if (real instanceof HybridInstance) { + crud.reloadQuietly(real.getName()); + } } } } @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException { + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { if ("hybrid".equals(entity)) { String hybridName = cacheKey; - if (event == Event.DROP) - hybridMap.removeLocal(hybridName); - else - reloadHybridInstance(hybridName); + try (AutoLock l = lock.lockForWrite()) { + if (event == Event.DROP) + hybridMap.removeLocal(hybridName); + else + crud.reloadQuietly(hybridName); + } - for (ProjectInstance prj : ProjectManager.getInstance(config).findProjects(RealizationType.HYBRID, hybridName)) { + for (ProjectInstance prj : ProjectManager.getInstance(config).findProjects(RealizationType.HYBRID, + hybridName)) { broadcaster.notifyProjectSchemaUpdate(prj.getName()); } } else if ("cube".equals(entity)) { String cubeName = cacheKey; - for (HybridInstance hybrid : getHybridInstancesByChild(RealizationType.CUBE, cubeName)) { - reloadHybridInstance(hybrid.getName()); + try (AutoLock l = lock.lockForWrite()) { + for (HybridInstance hybrid : getHybridInstancesByChild(RealizationType.CUBE, cubeName)) { + crud.reloadQuietly(hybrid.getName()); + } } } } } - public void reloadAllHybridInstance() throws IOException { - ResourceStore store = getStore(); - List paths = store.collectResourceRecursively(ResourceStore.HYBRID_RESOURCE_ROOT, ".json"); - - hybridMap.clear(); - logger.debug("Loading Hybrid from folder " + store.getReadableResourcePath(ResourceStore.HYBRID_RESOURCE_ROOT)); - - for (String path : paths) { - reloadHybridInstanceAt(path); - } - - logger.debug("Loaded " + paths.size() + " Hybrid(s)"); - } - public List getHybridInstancesByChild(RealizationType type, String realizationName) { - List result = Lists.newArrayList(); - for (HybridInstance hybridInstance : hybridMap.values()) { - for (RealizationEntry realizationEntry : hybridInstance.getRealizationEntries()) { - if (realizationEntry.getType() == type && realizationEntry.getRealization().equalsIgnoreCase(realizationName)) { - result.add(hybridInstance); + try (AutoLock l = lock.lockForRead()) { + List result = Lists.newArrayList(); + for (HybridInstance hybridInstance : hybridMap.values()) { + for (RealizationEntry realizationEntry : hybridInstance.getRealizationEntries()) { + if (realizationEntry.getType() == type + && realizationEntry.getRealization().equalsIgnoreCase(realizationName)) { + result.add(hybridInstance); + } } - } - - } - - return result; - } - public void reloadHybridInstance(String name) { - reloadHybridInstanceAt(HybridInstance.concatResourcePath(name)); - } - - private synchronized HybridInstance reloadHybridInstanceAt(String path) { - ResourceStore store = getStore(); - - HybridInstance hybridInstance = null; - try { - hybridInstance = store.getResource(path, HybridInstance.class, HYBRID_SERIALIZER); - hybridInstance.setConfig(config); - - if (hybridInstance.getRealizationEntries() == null || hybridInstance.getRealizationEntries().size() == 0) { - throw new IllegalStateException("HybridInstance must have realization entries, " + path); } - if (StringUtils.isBlank(hybridInstance.getName())) - throw new IllegalStateException("HybridInstance name must not be blank, at " + path); - - final String name = hybridInstance.getName(); - hybridMap.putLocal(name, hybridInstance); - - return hybridInstance; - } catch (Exception e) { - logger.error("Error during load hybrid instance " + path, e); - return null; + return result; } } @@ -173,11 +154,27 @@ public IRealization getRealization(String name) { } public Collection listHybridInstances() { - return hybridMap.values(); + try (AutoLock l = lock.lockForRead()) { + return hybridMap.values(); + } } public HybridInstance getHybridInstance(String name) { - return hybridMap.get(name); + try (AutoLock l = lock.lockForRead()) { + return hybridMap.get(name); + } + } + + public HybridInstance reloadHybridInstance(String name) { + try (AutoLock l = lock.lockForWrite()) { + return crud.reload(name); + } + } + + public void reloadAllHybridInstance() throws IOException { + try (AutoLock l = lock.lockForWrite()) { + crud.reloadAll(); + } } private ResourceStore getStore() { diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java index c8aee5df932..872deede16f 100644 --- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java +++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java @@ -182,7 +182,7 @@ public boolean accept(File pathname) { private void writeDictionary(Dataset intermediateTable, String cubeName, String segmentId) throws Exception { final KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv(); final CubeManager cubeManager = CubeManager.getInstance(kylinConfig); - final CubeInstance cubeInstance = cubeManager.reloadCubeLocal(cubeName); + final CubeInstance cubeInstance = cubeManager.getCube(cubeName); final String[] columns = intermediateTable.columns(); final CubeSegment seg = cubeInstance.getSegmentById(segmentId); final CubeDesc cubeDesc = cubeInstance.getDescriptor(); @@ -249,7 +249,7 @@ public void remove() { } private Map sampling(final JavaRDD> rowJavaRDD, final String cubeName, String segmentId) throws Exception { - CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).reloadCubeLocal(cubeName); + CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName); CubeSegment cubeSegment = cubeInstance.getSegmentById(segmentId); CubeDesc cubeDesc = cubeInstance.getDescriptor(); CuboidScheduler cuboidScheduler = cubeInstance.getCuboidScheduler(); diff --git a/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table1.json b/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table1.json index 8d534e85c90..3bf27f9cef3 100644 --- a/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table1.json +++ b/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table1.json @@ -1,4 +1,5 @@ { + "uuid" : "da93ed8c-aed7-4a98-ba05-28c89cfa8ee2", "session_id" : "DEFAULT_SESSION", "statement_id": "temp_table1", "statement": "as (select * from fact_table1)" diff --git a/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table2.json b/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table2.json index f4f5fadfd77..b2b6a797212 100644 --- a/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table2.json +++ b/examples/test_case_data/localmeta/temp_statement/DEFAULT_SESSION/temp_table2.json @@ -1,4 +1,5 @@ { + "uuid" : "9f37ed8c-aed7-4a98-ba05-28c89cfac870", "session_id" : "DEFAULT_SESSION", "statement_id": "temp_table2", "statement": "as (select * from fact_table2)" diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java index 239b4af6379..d338332defa 100644 --- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java +++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java @@ -131,7 +131,7 @@ public void before() throws Exception { BrokerConfig brokerConfig = kafkaConfig.getKafkaClusterConfigs().get(0).getBrokerConfigs().get(0); brokerConfig.setHost(localIp); kafkaConfig.setTopic(topicName); - KafkaConfigManager.getInstance(kylinConfig).saveKafkaConfig(kafkaConfig); + KafkaConfigManager.getInstance(kylinConfig).updateKafkaConfig(kafkaConfig); startEmbeddedKafka(topicName, brokerConfig); } diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/ModelController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/ModelController.java index afa914c87ee..cc4b7366c22 100644 --- a/server-base/src/main/java/org/apache/kylin/rest/controller/ModelController.java +++ b/server-base/src/main/java/org/apache/kylin/rest/controller/ModelController.java @@ -206,7 +206,7 @@ public ModelRequest cloneModel(@PathVariable String modelName, @RequestBody Mode newModelDesc = modelService.createModelDesc(project, newModelDesc); //reload avoid shallow - metaManager.reloadDataModelDescAt(DataModelDesc.concatResourcePath(newModelName)); + metaManager.reloadDataModel(newModelName); } catch (IOException e) { throw new InternalErrorException("failed to clone DataModelDesc", e); } diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/KafkaConfigService.java b/server-base/src/main/java/org/apache/kylin/rest/service/KafkaConfigService.java index 260cbc07f77..32c7339ba25 100644 --- a/server-base/src/main/java/org/apache/kylin/rest/service/KafkaConfigService.java +++ b/server-base/src/main/java/org/apache/kylin/rest/service/KafkaConfigService.java @@ -75,7 +75,7 @@ public KafkaConfig createKafkaConfig(KafkaConfig config, String project) throws if (getKafkaManager().getKafkaConfig(config.getName()) != null) { throw new BadRequestException(String.format(msg.getKAFKA_CONFIG_ALREADY_EXIST(), config.getName())); } - getKafkaManager().createKafkaConfig(config.getName(), config); + getKafkaManager().createKafkaConfig(config); return config; } diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/StreamingService.java b/server-base/src/main/java/org/apache/kylin/rest/service/StreamingService.java index 1f907f81c4c..d4d7cc71f13 100644 --- a/server-base/src/main/java/org/apache/kylin/rest/service/StreamingService.java +++ b/server-base/src/main/java/org/apache/kylin/rest/service/StreamingService.java @@ -41,7 +41,7 @@ public List listAllStreamingConfigs(final String table) throws if (StringUtils.isEmpty(table)) { streamingConfigs = getStreamingManager().listAllStreaming(); } else { - StreamingConfig config = getStreamingManager().getConfig(table); + StreamingConfig config = getStreamingManager().getStreamingConfig(table); if (config != null) { streamingConfigs.add(config); } @@ -73,7 +73,7 @@ public StreamingConfig createStreamingConfig(StreamingConfig config, String proj if (getStreamingManager().getStreamingConfig(config.getName()) != null) { throw new BadRequestException(String.format(msg.getSTREAMING_CONFIG_ALREADY_EXIST(), config.getName())); } - StreamingConfig streamingConfig = getStreamingManager().saveStreamingConfig(config); + StreamingConfig streamingConfig = getStreamingManager().createStreamingConfig(config); return streamingConfig; } diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java index 5e451b4fccc..ad6a46a0c8e 100644 --- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java +++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java @@ -24,12 +24,12 @@ import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.JsonSerializer; import org.apache.kylin.common.persistence.ResourceStore; -import org.apache.kylin.common.persistence.Serializer; -import org.apache.kylin.metadata.MetadataConstants; +import org.apache.kylin.common.util.AutoReadWriteLock; +import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock; import org.apache.kylin.metadata.cachesync.Broadcaster; import org.apache.kylin.metadata.cachesync.Broadcaster.Event; +import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache; import org.apache.kylin.source.kafka.config.KafkaConfig; import org.slf4j.Logger; @@ -39,10 +39,9 @@ */ public class KafkaConfigManager { + @SuppressWarnings("unused") private static final Logger logger = LoggerFactory.getLogger(KafkaConfigManager.class); - public static final Serializer KAFKA_SERIALIZER = new JsonSerializer(KafkaConfig.class); - public static KafkaConfigManager getInstance(KylinConfig config) { return config.getManager(KafkaConfigManager.class); } @@ -55,27 +54,39 @@ static KafkaConfigManager newInstance(KylinConfig config) throws IOException { // ============================================================================ private KylinConfig config; - + // name ==> StreamingConfig private CaseInsensitiveStringCache kafkaMap; - + private CachedCrudAssist crud; + private AutoReadWriteLock lock = new AutoReadWriteLock(); + private KafkaConfigManager(KylinConfig config) throws IOException { this.config = config; this.kafkaMap = new CaseInsensitiveStringCache(config, "kafka"); + this.crud = new CachedCrudAssist(getStore(), ResourceStore.KAFKA_RESOURCE_ROOT, KafkaConfig.class, + kafkaMap) { + @Override + protected KafkaConfig initEntityAfterReload(KafkaConfig t, String resourceName) { + return t; // noop + } + }; // touch lower level metadata before registering my listener - reloadAllKafkaConfig(); + crud.reloadAll(); Broadcaster.getInstance(config).registerListener(new KafkaSyncListener(), "kafka"); } private class KafkaSyncListener extends Broadcaster.Listener { @Override - public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException { - if (event == Event.DROP) - removeKafkaConfigLocal(cacheKey); - else - reloadKafkaConfigLocal(cacheKey); + public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) + throws IOException { + try (AutoLock l = lock.lockForWrite()) { + if (event == Event.DROP) + kafkaMap.removeLocal(cacheKey); + else + crud.reloadQuietly(cacheKey); + } } } @@ -83,85 +94,45 @@ private ResourceStore getStore() { return ResourceStore.getStore(this.config); } - public List listAllKafkaConfigs() { - return new ArrayList(kafkaMap.values()); + public KafkaConfig getKafkaConfig(String name) { + try (AutoLock l = lock.lockForRead()) { + return kafkaMap.get(name); + } } - /** - * Reload KafkaConfig from resource store It will be triggered by an desc - * update event. - * - * @param name - * @throws IOException - */ - public KafkaConfig reloadKafkaConfigLocal(String name) throws IOException { - - // Save Source - String path = KafkaConfig.concatResourcePath(name); - - // Reload the KafkaConfig - KafkaConfig ndesc = loadKafkaConfigAt(path); - - // Here replace the old one - kafkaMap.putLocal(ndesc.getName(), ndesc); - return ndesc; + public List listAllKafkaConfigs() { + try (AutoLock l = lock.lockForRead()) { + return new ArrayList(kafkaMap.values()); + } } - public boolean createKafkaConfig(String name, KafkaConfig config) { + public boolean createKafkaConfig(KafkaConfig kafkaConfig) throws IOException { + try (AutoLock l = lock.lockForWrite()) { - if (config == null || StringUtils.isEmpty(config.getName())) { - throw new IllegalArgumentException(); - } + if (kafkaMap.containsKey(kafkaConfig.resourceName())) + throw new IllegalArgumentException("KafkaConfig '" + kafkaConfig.getName() + "' already exists"); + + kafkaConfig.updateRandomUuid(); + checkKafkaConfig(kafkaConfig); - if (kafkaMap.containsKey(config.getName())) - throw new IllegalArgumentException("KafkaConfig '" + config.getName() + "' already exists"); - try { - getStore().putResource(KafkaConfig.concatResourcePath(name), config, KafkaConfig.SERIALIZER); - kafkaMap.put(config.getName(), config); + crud.save(kafkaConfig); return true; - } catch (IOException e) { - logger.error("error save resource name:" + name, e); - throw new RuntimeException("error save resource name:" + name, e); } } - public KafkaConfig updateKafkaConfig(KafkaConfig desc) throws IOException { - // Validate KafkaConfig - if (desc.getUuid() == null || desc.getName() == null) { - throw new IllegalArgumentException(); - } - String name = desc.getName(); - if (!kafkaMap.containsKey(name)) { - throw new IllegalArgumentException("KafkaConfig '" + name + "' does not exist."); - } - - // Save Source - String path = desc.getResourcePath(); - getStore().putResource(path, desc, KAFKA_SERIALIZER); + public KafkaConfig updateKafkaConfig(KafkaConfig kafkaConfig) throws IOException { + try (AutoLock l = lock.lockForWrite()) { - // Reload the KafkaConfig - KafkaConfig ndesc = loadKafkaConfigAt(path); - // Here replace the old one - kafkaMap.put(ndesc.getName(), desc); + if (!kafkaMap.containsKey(kafkaConfig.resourceName())) + throw new IllegalArgumentException("KafkaConfig '" + kafkaConfig.getName() + "' does not exist."); - return ndesc; - } - - private KafkaConfig loadKafkaConfigAt(String path) throws IOException { - ResourceStore store = getStore(); - KafkaConfig kafkaConfig = store.getResource(path, KafkaConfig.class, KAFKA_SERIALIZER); - - if (StringUtils.isBlank(kafkaConfig.getName())) { - throw new IllegalStateException("KafkaConfig name must not be blank"); + checkKafkaConfig(kafkaConfig); + + return crud.save(kafkaConfig); } - return kafkaConfig; } - public KafkaConfig getKafkaConfig(String name) { - return kafkaMap.get(name); - } - - public void saveKafkaConfig(KafkaConfig kafkaConfig) throws IOException { + private void checkKafkaConfig(KafkaConfig kafkaConfig) { if (kafkaConfig == null || StringUtils.isEmpty(kafkaConfig.getName())) { throw new IllegalArgumentException(); } @@ -173,50 +144,13 @@ public void saveKafkaConfig(KafkaConfig kafkaConfig) throws IOException { if (kafkaConfig.getKafkaClusterConfigs() == null || kafkaConfig.getKafkaClusterConfigs().size() == 0) { throw new IllegalArgumentException("No cluster info"); } - - String path = KafkaConfig.concatResourcePath(kafkaConfig.getName()); - getStore().putResource(path, kafkaConfig, KafkaConfig.SERIALIZER); } // remove kafkaConfig public void removeKafkaConfig(KafkaConfig kafkaConfig) throws IOException { - String path = kafkaConfig.getResourcePath(); - getStore().deleteResource(path); - kafkaMap.remove(kafkaConfig.getName()); - } - - private void removeKafkaConfigLocal(String name) { - kafkaMap.remove(name); - } - - private void reloadAllKafkaConfig() throws IOException { - ResourceStore store = getStore(); - logger.info("Reloading Kafka Metadata from folder " + store.getReadableResourcePath(ResourceStore.KAFKA_RESOURCE_ROOT)); - - kafkaMap.clear(); - - List paths = store.collectResourceRecursively(ResourceStore.KAFKA_RESOURCE_ROOT, MetadataConstants.FILE_SURFIX); - for (String path : paths) { - KafkaConfig kafkaConfig; - try { - kafkaConfig = loadKafkaConfigAt(path); - } catch (Exception e) { - logger.error("Error loading kafkaConfig desc " + path, e); - continue; - } - if (path.equals(kafkaConfig.getResourcePath()) == false) { - logger.error("Skip suspicious desc at " + path + ", " + kafkaConfig + " should be at " + kafkaConfig.getResourcePath()); - continue; - } - if (kafkaMap.containsKey(kafkaConfig.getName())) { - logger.error("Dup KafkaConfig name '" + kafkaConfig.getName() + "' on path " + path); - continue; - } - - kafkaMap.putLocal(kafkaConfig.getName(), kafkaConfig); + try (AutoLock l = lock.lockForWrite()) { + crud.delete(kafkaConfig); } - - logger.debug("Loaded " + kafkaMap.size() + " KafkaConfig(s)"); } } diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java index 82b89028362..696c20c20d9 100644 --- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java +++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java @@ -30,14 +30,15 @@ import org.apache.kylin.common.persistence.RootPersistentEntity; import org.apache.kylin.common.persistence.Serializer; import org.apache.kylin.metadata.MetadataConstants; +import org.apache.kylin.source.kafka.TimedJsonStreamParser; import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonManagedReference; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.kylin.source.kafka.TimedJsonStreamParser; /** */ +@SuppressWarnings("serial") @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) public class KafkaConfig extends RootPersistentEntity { @@ -70,6 +71,11 @@ public class KafkaConfig extends RootPersistentEntity { @JsonProperty("parserProperties") private String parserProperties; + @Override + public String resourceName() { + return name; + } + public String getResourcePath() { return concatResourcePath(name); }